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     */
018    package org.apache.hadoop.hdfs.server.namenode;
019    
020    import java.io.PrintStream;
021    import java.io.PrintWriter;
022    import java.io.StringWriter;
023    import java.util.ArrayList;
024    import java.util.List;
025    
026    import org.apache.commons.logging.Log;
027    import org.apache.commons.logging.LogFactory;
028    import org.apache.hadoop.classification.InterfaceAudience;
029    import org.apache.hadoop.fs.ContentSummary;
030    import org.apache.hadoop.fs.Path;
031    import org.apache.hadoop.fs.permission.FsPermission;
032    import org.apache.hadoop.fs.permission.PermissionStatus;
033    import org.apache.hadoop.hdfs.DFSUtil;
034    import org.apache.hadoop.hdfs.protocol.Block;
035    import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
036    import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
037    import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
038    import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
039    import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
040    import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
041    import org.apache.hadoop.hdfs.util.Diff;
042    import org.apache.hadoop.util.StringUtils;
043    
044    import com.google.common.annotations.VisibleForTesting;
045    import com.google.common.base.Preconditions;
046    
047    /**
048     * We keep an in-memory representation of the file/block hierarchy.
049     * This is a base INode class containing common fields for file and 
050     * directory inodes.
051     */
052    @InterfaceAudience.Private
053    public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
054      public static final Log LOG = LogFactory.getLog(INode.class);
055    
056      /** parent is either an {@link INodeDirectory} or an {@link INodeReference}.*/
057      private INode parent = null;
058    
059      INode(INode parent) {
060        this.parent = parent;
061      }
062    
063      /** Get inode id */
064      public abstract long getId();
065    
066      /**
067       * Check whether this is the root inode.
068       */
069      final boolean isRoot() {
070        return getLocalNameBytes().length == 0;
071      }
072    
073      /** Get the {@link PermissionStatus} */
074      abstract PermissionStatus getPermissionStatus(Snapshot snapshot);
075    
076      /** The same as getPermissionStatus(null). */
077      final PermissionStatus getPermissionStatus() {
078        return getPermissionStatus(null);
079      }
080    
081      /**
082       * @param snapshot
083       *          if it is not null, get the result from the given snapshot;
084       *          otherwise, get the result from the current inode.
085       * @return user name
086       */
087      abstract String getUserName(Snapshot snapshot);
088    
089      /** The same as getUserName(null). */
090      @Override
091      public final String getUserName() {
092        return getUserName(null);
093      }
094    
095      /** Set user */
096      abstract void setUser(String user);
097    
098      /** Set user */
099      final INode setUser(String user, Snapshot latest, INodeMap inodeMap)
100          throws QuotaExceededException {
101        final INode nodeToUpdate = recordModification(latest, inodeMap);
102        nodeToUpdate.setUser(user);
103        return nodeToUpdate;
104      }
105      /**
106       * @param snapshot
107       *          if it is not null, get the result from the given snapshot;
108       *          otherwise, get the result from the current inode.
109       * @return group name
110       */
111      abstract String getGroupName(Snapshot snapshot);
112    
113      /** The same as getGroupName(null). */
114      @Override
115      public final String getGroupName() {
116        return getGroupName(null);
117      }
118    
119      /** Set group */
120      abstract void setGroup(String group);
121    
122      /** Set group */
123      final INode setGroup(String group, Snapshot latest, INodeMap inodeMap)
124          throws QuotaExceededException {
125        final INode nodeToUpdate = recordModification(latest, inodeMap);
126        nodeToUpdate.setGroup(group);
127        return nodeToUpdate;
128      }
129    
130      /**
131       * @param snapshot
132       *          if it is not null, get the result from the given snapshot;
133       *          otherwise, get the result from the current inode.
134       * @return permission.
135       */
136      abstract FsPermission getFsPermission(Snapshot snapshot);
137      
138      /** The same as getFsPermission(null). */
139      @Override
140      public final FsPermission getFsPermission() {
141        return getFsPermission(null);
142      }
143    
144      /** Set the {@link FsPermission} of this {@link INode} */
145      abstract void setPermission(FsPermission permission);
146    
147      /** Set the {@link FsPermission} of this {@link INode} */
148      INode setPermission(FsPermission permission, Snapshot latest,
149          INodeMap inodeMap) throws QuotaExceededException {
150        final INode nodeToUpdate = recordModification(latest, inodeMap);
151        nodeToUpdate.setPermission(permission);
152        return nodeToUpdate;
153      }
154    
155      /**
156       * @return if the given snapshot is null, return this;
157       *     otherwise return the corresponding snapshot inode.
158       */
159      public INodeAttributes getSnapshotINode(final Snapshot snapshot) {
160        return this;
161      }
162    
163      /** Is this inode in the latest snapshot? */
164      public final boolean isInLatestSnapshot(final Snapshot latest) {
165        if (latest == null) {
166          return false;
167        }
168        // if parent is a reference node, parent must be a renamed node. We can 
169        // stop the check at the reference node.
170        if (parent != null && parent.isReference()) {
171          return true;
172        }
173        final INodeDirectory parentDir = getParent();
174        if (parentDir == null) { // root
175          return true;
176        }
177        if (!parentDir.isInLatestSnapshot(latest)) {
178          return false;
179        }
180        final INode child = parentDir.getChild(getLocalNameBytes(), latest);
181        if (this == child) {
182          return true;
183        }
184        if (child == null || !(child.isReference())) {
185          return false;
186        }
187        return this == child.asReference().getReferredINode();
188      }
189      
190      /** @return true if the given inode is an ancestor directory of this inode. */
191      public final boolean isAncestorDirectory(final INodeDirectory dir) {
192        for(INodeDirectory p = getParent(); p != null; p = p.getParent()) {
193          if (p == dir) {
194            return true;
195          }
196        }
197        return false;
198      }
199    
200      /**
201       * When {@link #recordModification} is called on a referred node,
202       * this method tells which snapshot the modification should be
203       * associated with: the snapshot that belongs to the SRC tree of the rename
204       * operation, or the snapshot belonging to the DST tree.
205       * 
206       * @param latestInDst
207       *          the latest snapshot in the DST tree above the reference node
208       * @return True: the modification should be recorded in the snapshot that
209       *         belongs to the SRC tree. False: the modification should be
210       *         recorded in the snapshot that belongs to the DST tree.
211       */
212      public final boolean shouldRecordInSrcSnapshot(final Snapshot latestInDst) {
213        Preconditions.checkState(!isReference());
214    
215        if (latestInDst == null) {
216          return true;
217        }
218        INodeReference withCount = getParentReference();
219        if (withCount != null) {
220          int dstSnapshotId = withCount.getParentReference().getDstSnapshotId();
221          if (dstSnapshotId >= latestInDst.getId()) {
222            return true;
223          }
224        }
225        return false;
226      }
227    
228      /**
229       * This inode is being modified.  The previous version of the inode needs to
230       * be recorded in the latest snapshot.
231       *
232       * @param latest the latest snapshot that has been taken.
233       *        Note that it is null if no snapshots have been taken.
234       * @param inodeMap while recording modification, the inode or its parent may 
235       *                 get replaced, and the inodeMap needs to be updated.
236       * @return The current inode, which usually is the same object of this inode.
237       *         However, in some cases, this inode may be replaced with a new inode
238       *         for maintaining snapshots. The current inode is then the new inode.
239       */
240      abstract INode recordModification(final Snapshot latest,
241          final INodeMap inodeMap) throws QuotaExceededException;
242    
243      /** Check whether it's a reference. */
244      public boolean isReference() {
245        return false;
246      }
247    
248      /** Cast this inode to an {@link INodeReference}.  */
249      public INodeReference asReference() {
250        throw new IllegalStateException("Current inode is not a reference: "
251            + this.toDetailString());
252      }
253    
254      /**
255       * Check whether it's a file.
256       */
257      public boolean isFile() {
258        return false;
259      }
260    
261      /** Cast this inode to an {@link INodeFile}.  */
262      public INodeFile asFile() {
263        throw new IllegalStateException("Current inode is not a file: "
264            + this.toDetailString());
265      }
266    
267      /**
268       * Check whether it's a directory
269       */
270      public boolean isDirectory() {
271        return false;
272      }
273    
274      /** Cast this inode to an {@link INodeDirectory}.  */
275      public INodeDirectory asDirectory() {
276        throw new IllegalStateException("Current inode is not a directory: "
277            + this.toDetailString());
278      }
279    
280      /**
281       * Check whether it's a symlink
282       */
283      public boolean isSymlink() {
284        return false;
285      }
286    
287      /** Cast this inode to an {@link INodeSymlink}.  */
288      public INodeSymlink asSymlink() {
289        throw new IllegalStateException("Current inode is not a symlink: "
290            + this.toDetailString());
291      }
292    
293      /**
294       * Clean the subtree under this inode and collect the blocks from the descents
295       * for further block deletion/update. The current inode can either resides in
296       * the current tree or be stored as a snapshot copy.
297       * 
298       * <pre>
299       * In general, we have the following rules. 
300       * 1. When deleting a file/directory in the current tree, we have different 
301       * actions according to the type of the node to delete. 
302       * 
303       * 1.1 The current inode (this) is an {@link INodeFile}. 
304       * 1.1.1 If {@code prior} is null, there is no snapshot taken on ancestors 
305       * before. Thus we simply destroy (i.e., to delete completely, no need to save 
306       * snapshot copy) the current INode and collect its blocks for further 
307       * cleansing.
308       * 1.1.2 Else do nothing since the current INode will be stored as a snapshot
309       * copy.
310       * 
311       * 1.2 The current inode is an {@link INodeDirectory}.
312       * 1.2.1 If {@code prior} is null, there is no snapshot taken on ancestors 
313       * before. Similarly, we destroy the whole subtree and collect blocks.
314       * 1.2.2 Else do nothing with the current INode. Recursively clean its 
315       * children.
316       * 
317       * 1.3 The current inode is a {@link FileWithSnapshot}.
318       * Call recordModification(..) to capture the current states.
319       * Mark the INode as deleted.
320       * 
321       * 1.4 The current inode is a {@link INodeDirectoryWithSnapshot}.
322       * Call recordModification(..) to capture the current states. 
323       * Destroy files/directories created after the latest snapshot 
324       * (i.e., the inodes stored in the created list of the latest snapshot).
325       * Recursively clean remaining children. 
326       *
327       * 2. When deleting a snapshot.
328       * 2.1 To clean {@link INodeFile}: do nothing.
329       * 2.2 To clean {@link INodeDirectory}: recursively clean its children.
330       * 2.3 To clean {@link FileWithSnapshot}: delete the corresponding snapshot in
331       * its diff list.
332       * 2.4 To clean {@link INodeDirectoryWithSnapshot}: delete the corresponding 
333       * snapshot in its diff list. Recursively clean its children.
334       * </pre>
335       * 
336       * @param snapshot
337       *          The snapshot to delete. Null means to delete the current
338       *          file/directory.
339       * @param prior
340       *          The latest snapshot before the to-be-deleted snapshot. When
341       *          deleting a current inode, this parameter captures the latest
342       *          snapshot.
343       * @param collectedBlocks
344       *          blocks collected from the descents for further block
345       *          deletion/update will be added to the given map.
346       * @param removedINodes
347       *          INodes collected from the descents for further cleaning up of 
348       *          inodeMap
349       * @return quota usage delta when deleting a snapshot
350       */
351      public abstract Quota.Counts cleanSubtree(final Snapshot snapshot,
352          Snapshot prior, BlocksMapUpdateInfo collectedBlocks,
353          List<INode> removedINodes, boolean countDiffChange)
354          throws QuotaExceededException;
355      
356      /**
357       * Destroy self and clear everything! If the INode is a file, this method
358       * collects its blocks for further block deletion. If the INode is a
359       * directory, the method goes down the subtree and collects blocks from the
360       * descents, and clears its parent/children references as well. The method
361       * also clears the diff list if the INode contains snapshot diff list.
362       * 
363       * @param collectedBlocks
364       *          blocks collected from the descents for further block
365       *          deletion/update will be added to this map.
366       * @param removedINodes
367       *          INodes collected from the descents for further cleaning up of
368       *          inodeMap
369       */
370      public abstract void destroyAndCollectBlocks(
371          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
372    
373      /** Compute {@link ContentSummary}. */
374      public final ContentSummary computeContentSummary() {
375        final Content.Counts counts = computeContentSummary(
376            Content.Counts.newInstance());
377        return new ContentSummary(counts.get(Content.LENGTH),
378            counts.get(Content.FILE) + counts.get(Content.SYMLINK),
379            counts.get(Content.DIRECTORY), getNsQuota(),
380            counts.get(Content.DISKSPACE), getDsQuota());
381      }
382    
383      /**
384       * Count subtree content summary with a {@link Content.Counts}.
385       *
386       * @param counts The subtree counts for returning.
387       * @return The same objects as the counts parameter.
388       */
389      public abstract Content.Counts computeContentSummary(Content.Counts counts);
390      
391      /**
392       * Check and add namespace/diskspace consumed to itself and the ancestors.
393       * @throws QuotaExceededException if quote is violated.
394       */
395      public void addSpaceConsumed(long nsDelta, long dsDelta, boolean verify) 
396          throws QuotaExceededException {
397        if (parent != null) {
398          parent.addSpaceConsumed(nsDelta, dsDelta, verify);
399        }
400      }
401    
402      /**
403       * Get the quota set for this inode
404       * @return the quota if it is set; -1 otherwise
405       */
406      public long getNsQuota() {
407        return -1;
408      }
409    
410      public long getDsQuota() {
411        return -1;
412      }
413      
414      public final boolean isQuotaSet() {
415        return getNsQuota() >= 0 || getDsQuota() >= 0;
416      }
417      
418      /**
419       * Count subtree {@link Quota#NAMESPACE} and {@link Quota#DISKSPACE} usages.
420       */
421      public final Quota.Counts computeQuotaUsage() {
422        return computeQuotaUsage(new Quota.Counts(), true);
423      }
424    
425      /**
426       * Count subtree {@link Quota#NAMESPACE} and {@link Quota#DISKSPACE} usages.
427       * 
428       * With the existence of {@link INodeReference}, the same inode and its
429       * subtree may be referred by multiple {@link WithName} nodes and a
430       * {@link DstReference} node. To avoid circles while quota usage computation,
431       * we have the following rules:
432       * 
433       * <pre>
434       * 1. For a {@link DstReference} node, since the node must be in the current
435       * tree (or has been deleted as the end point of a series of rename 
436       * operations), we compute the quota usage of the referred node (and its 
437       * subtree) in the regular manner, i.e., including every inode in the current
438       * tree and in snapshot copies, as well as the size of diff list.
439       * 
440       * 2. For a {@link WithName} node, since the node must be in a snapshot, we 
441       * only count the quota usage for those nodes that still existed at the 
442       * creation time of the snapshot associated with the {@link WithName} node.
443       * We do not count in the size of the diff list.  
444       * <pre>
445       * 
446       * @param counts The subtree counts for returning.
447       * @param useCache Whether to use cached quota usage. Note that 
448       *                 {@link WithName} node never uses cache for its subtree.
449       * @param lastSnapshotId {@link Snapshot#INVALID_ID} indicates the computation
450       *                       is in the current tree. Otherwise the id indicates
451       *                       the computation range for a {@link WithName} node.
452       * @return The same objects as the counts parameter.
453       */
454      public abstract Quota.Counts computeQuotaUsage(Quota.Counts counts,
455          boolean useCache, int lastSnapshotId);
456    
457      public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
458          boolean useCache) {
459        return computeQuotaUsage(counts, useCache, Snapshot.INVALID_ID);
460      }
461      
462      /**
463       * @return null if the local name is null; otherwise, return the local name.
464       */
465      public final String getLocalName() {
466        final byte[] name = getLocalNameBytes();
467        return name == null? null: DFSUtil.bytes2String(name);
468      }
469    
470      @Override
471      public final byte[] getKey() {
472        return getLocalNameBytes();
473      }
474    
475      /**
476       * Set local file name
477       */
478      public abstract void setLocalName(byte[] name);
479    
480      public String getFullPathName() {
481        // Get the full path name of this inode.
482        return FSDirectory.getFullPathName(this);
483      }
484      
485      @Override
486      public String toString() {
487        return getLocalName();
488      }
489    
490      @VisibleForTesting
491      public final String getObjectString() {
492        return getClass().getSimpleName() + "@"
493            + Integer.toHexString(super.hashCode());
494      }
495    
496      /** @return a string description of the parent. */
497      @VisibleForTesting
498      public final String getParentString() {
499        final INodeReference parentRef = getParentReference();
500        if (parentRef != null) {
501          return "parentRef=" + parentRef.getLocalName() + "->";
502        } else {
503          final INodeDirectory parentDir = getParent();
504          if (parentDir != null) {
505            return "parentDir=" + parentDir.getLocalName() + "/";
506          } else {
507            return "parent=null";
508          }
509        }
510      }
511    
512      @VisibleForTesting
513      public String toDetailString() {
514        return toString() + "(" + getObjectString() + "), " + getParentString();
515      }
516    
517      /** @return the parent directory */
518      public final INodeDirectory getParent() {
519        return parent == null? null
520            : parent.isReference()? getParentReference().getParent(): parent.asDirectory();
521      }
522    
523      /**
524       * @return the parent as a reference if this is a referred inode;
525       *         otherwise, return null.
526       */
527      public INodeReference getParentReference() {
528        return parent == null || !parent.isReference()? null: (INodeReference)parent;
529      }
530    
531      /** Set parent directory */
532      public final void setParent(INodeDirectory parent) {
533        this.parent = parent;
534      }
535    
536      /** Set container. */
537      public final void setParentReference(INodeReference parent) {
538        this.parent = parent;
539      }
540    
541      /** Clear references to other objects. */
542      public void clear() {
543        setParent(null);
544      }
545    
546      /**
547       * @param snapshot
548       *          if it is not null, get the result from the given snapshot;
549       *          otherwise, get the result from the current inode.
550       * @return modification time.
551       */
552      abstract long getModificationTime(Snapshot snapshot);
553    
554      /** The same as getModificationTime(null). */
555      @Override
556      public final long getModificationTime() {
557        return getModificationTime(null);
558      }
559    
560      /** Update modification time if it is larger than the current value. */
561      public abstract INode updateModificationTime(long mtime, Snapshot latest,
562          INodeMap inodeMap) throws QuotaExceededException;
563    
564      /** Set the last modification time of inode. */
565      public abstract void setModificationTime(long modificationTime);
566    
567      /** Set the last modification time of inode. */
568      public final INode setModificationTime(long modificationTime,
569          Snapshot latest, INodeMap inodeMap) throws QuotaExceededException {
570        final INode nodeToUpdate = recordModification(latest, inodeMap);
571        nodeToUpdate.setModificationTime(modificationTime);
572        return nodeToUpdate;
573      }
574    
575      /**
576       * @param snapshot
577       *          if it is not null, get the result from the given snapshot;
578       *          otherwise, get the result from the current inode.
579       * @return access time
580       */
581      abstract long getAccessTime(Snapshot snapshot);
582    
583      /** The same as getAccessTime(null). */
584      @Override
585      public final long getAccessTime() {
586        return getAccessTime(null);
587      }
588    
589      /**
590       * Set last access time of inode.
591       */
592      public abstract void setAccessTime(long accessTime);
593    
594      /**
595       * Set last access time of inode.
596       */
597      public final INode setAccessTime(long accessTime, Snapshot latest,
598          INodeMap inodeMap) throws QuotaExceededException {
599        final INode nodeToUpdate = recordModification(latest, inodeMap);
600        nodeToUpdate.setAccessTime(accessTime);
601        return nodeToUpdate;
602      }
603    
604    
605      /**
606       * Breaks file path into components.
607       * @param path
608       * @return array of byte arrays each of which represents 
609       * a single path component.
610       */
611      static byte[][] getPathComponents(String path) {
612        return getPathComponents(getPathNames(path));
613      }
614    
615      /** Convert strings to byte arrays for path components. */
616      static byte[][] getPathComponents(String[] strings) {
617        if (strings.length == 0) {
618          return new byte[][]{null};
619        }
620        byte[][] bytes = new byte[strings.length][];
621        for (int i = 0; i < strings.length; i++)
622          bytes[i] = DFSUtil.string2Bytes(strings[i]);
623        return bytes;
624      }
625    
626      /**
627       * Splits an absolute path into an array of path components.
628       * @param path
629       * @throws AssertionError if the given path is invalid.
630       * @return array of path components.
631       */
632      static String[] getPathNames(String path) {
633        if (path == null || !path.startsWith(Path.SEPARATOR)) {
634          throw new AssertionError("Absolute path required");
635        }
636        return StringUtils.split(path, Path.SEPARATOR_CHAR);
637      }
638    
639      @Override
640      public final int compareTo(byte[] bytes) {
641        return DFSUtil.compareBytes(getLocalNameBytes(), bytes);
642      }
643    
644      @Override
645      public final boolean equals(Object that) {
646        if (this == that) {
647          return true;
648        }
649        if (that == null || !(that instanceof INode)) {
650          return false;
651        }
652        return getId() == ((INode) that).getId();
653      }
654    
655      @Override
656      public final int hashCode() {
657        long id = getId();
658        return (int)(id^(id>>>32));  
659      }
660      
661      /**
662       * Dump the subtree starting from this inode.
663       * @return a text representation of the tree.
664       */
665      @VisibleForTesting
666      public final StringBuffer dumpTreeRecursively() {
667        final StringWriter out = new StringWriter(); 
668        dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder(), null);
669        return out.getBuffer();
670      }
671    
672      @VisibleForTesting
673      public final void dumpTreeRecursively(PrintStream out) {
674        dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder(), null);
675      }
676    
677      /**
678       * Dump tree recursively.
679       * @param prefix The prefix string that each line should print.
680       */
681      @VisibleForTesting
682      public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
683          Snapshot snapshot) {
684        out.print(prefix);
685        out.print(" ");
686        final String name = getLocalName();
687        out.print(name.isEmpty()? "/": name);
688        out.print("   (");
689        out.print(getObjectString());
690        out.print("), ");
691        out.print(getParentString());
692        out.print(", " + getPermissionStatus(snapshot));
693      }
694      
695      /**
696       * Information used for updating the blocksMap when deleting files.
697       */
698      public static class BlocksMapUpdateInfo {
699        /**
700         * The list of blocks that need to be removed from blocksMap
701         */
702        private List<Block> toDeleteList;
703        
704        public BlocksMapUpdateInfo(List<Block> toDeleteList) {
705          this.toDeleteList = toDeleteList == null ? new ArrayList<Block>()
706              : toDeleteList;
707        }
708        
709        public BlocksMapUpdateInfo() {
710          toDeleteList = new ArrayList<Block>();
711        }
712        
713        /**
714         * @return The list of blocks that need to be removed from blocksMap
715         */
716        public List<Block> getToDeleteList() {
717          return toDeleteList;
718        }
719        
720        /**
721         * Add a to-be-deleted block into the
722         * {@link BlocksMapUpdateInfo#toDeleteList}
723         * @param toDelete the to-be-deleted block
724         */
725        public void addDeleteBlock(Block toDelete) {
726          if (toDelete != null) {
727            toDeleteList.add(toDelete);
728          }
729        }
730        
731        /**
732         * Clear {@link BlocksMapUpdateInfo#toDeleteList}
733         */
734        public void clear() {
735          toDeleteList.clear();
736        }
737      }
738    }