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 java.util.Arrays;
021import java.util.Collections;
022import java.util.List;
023import java.util.NoSuchElementException;
024
025import com.google.common.collect.ImmutableList;
026import org.apache.commons.logging.Log;
027import org.apache.commons.logging.LogFactory;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.fs.UnresolvedLinkException;
030import org.apache.hadoop.hdfs.DFSUtil;
031import org.apache.hadoop.hdfs.protocol.HdfsConstants;
032import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
033import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
034import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
035
036import com.google.common.base.Preconditions;
037
038import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
039import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.ID_INTEGER_COMPARATOR;
040
041/**
042 * Contains INodes information resolved from a given path.
043 */
044public class INodesInPath {
045  public static final Log LOG = LogFactory.getLog(INodesInPath.class);
046
047  /**
048   * @return true if path component is {@link HdfsConstants#DOT_SNAPSHOT_DIR}
049   */
050  private static boolean isDotSnapshotDir(byte[] pathComponent) {
051    return pathComponent != null &&
052        Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
053  }
054
055  static INodesInPath fromINode(INode inode) {
056    int depth = 0, index;
057    INode tmp = inode;
058    while (tmp != null) {
059      depth++;
060      tmp = tmp.getParent();
061    }
062    final byte[][] path = new byte[depth][];
063    final INode[] inodes = new INode[depth];
064    tmp = inode;
065    index = depth;
066    while (tmp != null) {
067      index--;
068      path[index] = tmp.getKey();
069      inodes[index] = tmp;
070      tmp = tmp.getParent();
071    }
072    return new INodesInPath(inodes, path);
073  }
074
075  /**
076   * Given some components, create a path name.
077   * @param components The path components
078   * @param start index
079   * @param end index
080   * @return concatenated path
081   */
082  private static String constructPath(byte[][] components, int start, int end) {
083    StringBuilder buf = new StringBuilder();
084    for (int i = start; i < end; i++) {
085      buf.append(DFSUtil.bytes2String(components[i]));
086      if (i < end - 1) {
087        buf.append(Path.SEPARATOR);
088      }
089    }
090    return buf.toString();
091  }
092
093  /**
094   * Retrieve existing INodes from a path. For non-snapshot path,
095   * the number of INodes is equal to the number of path components. For
096   * snapshot path (e.g., /foo/.snapshot/s1/bar), the number of INodes is
097   * (number_of_path_components - 1).
098   * 
099   * An UnresolvedPathException is always thrown when an intermediate path 
100   * component refers to a symbolic link. If the final path component refers 
101   * to a symbolic link then an UnresolvedPathException is only thrown if
102   * resolveLink is true.  
103   * 
104   * <p>
105   * Example: <br>
106   * Given the path /c1/c2/c3 where only /c1/c2 exists, resulting in the
107   * following path components: ["","c1","c2","c3"]
108   * 
109   * <p>
110   * <code>getExistingPathINodes(["","c1","c2"])</code> should fill
111   * the array with [rootINode,c1,c2], <br>
112   * <code>getExistingPathINodes(["","c1","c2","c3"])</code> should
113   * fill the array with [rootINode,c1,c2,null]
114   * 
115   * @param startingDir the starting directory
116   * @param components array of path component name
117   * @param resolveLink indicates whether UnresolvedLinkException should
118   *        be thrown when the path refers to a symbolic link.
119   * @return the specified number of existing INodes in the path
120   */
121  static INodesInPath resolve(final INodeDirectory startingDir,
122      final byte[][] components, final boolean resolveLink)
123      throws UnresolvedLinkException {
124    Preconditions.checkArgument(startingDir.compareTo(components[0]) == 0);
125
126    INode curNode = startingDir;
127    int count = 0;
128    int inodeNum = 0;
129    INode[] inodes = new INode[components.length];
130    boolean isSnapshot = false;
131    int snapshotId = CURRENT_STATE_ID;
132
133    while (count < components.length && curNode != null) {
134      final boolean lastComp = (count == components.length - 1);
135      inodes[inodeNum++] = curNode;
136      final boolean isRef = curNode.isReference();
137      final boolean isDir = curNode.isDirectory();
138      final INodeDirectory dir = isDir? curNode.asDirectory(): null;
139      if (!isRef && isDir && dir.isWithSnapshot()) {
140        //if the path is a non-snapshot path, update the latest snapshot.
141        if (!isSnapshot && shouldUpdateLatestId(
142            dir.getDirectoryWithSnapshotFeature().getLastSnapshotId(),
143            snapshotId)) {
144          snapshotId = dir.getDirectoryWithSnapshotFeature().getLastSnapshotId();
145        }
146      } else if (isRef && isDir && !lastComp) {
147        // If the curNode is a reference node, need to check its dstSnapshot:
148        // 1. if the existing snapshot is no later than the dstSnapshot (which
149        // is the latest snapshot in dst before the rename), the changes 
150        // should be recorded in previous snapshots (belonging to src).
151        // 2. however, if the ref node is already the last component, we still 
152        // need to know the latest snapshot among the ref node's ancestors, 
153        // in case of processing a deletion operation. Thus we do not overwrite
154        // the latest snapshot if lastComp is true. In case of the operation is
155        // a modification operation, we do a similar check in corresponding 
156        // recordModification method.
157        if (!isSnapshot) {
158          int dstSnapshotId = curNode.asReference().getDstSnapshotId();
159          if (snapshotId == CURRENT_STATE_ID || // no snapshot in dst tree of rename
160              (dstSnapshotId != CURRENT_STATE_ID &&
161               dstSnapshotId >= snapshotId)) { // the above scenario
162            int lastSnapshot = CURRENT_STATE_ID;
163            DirectoryWithSnapshotFeature sf;
164            if (curNode.isDirectory() && 
165                (sf = curNode.asDirectory().getDirectoryWithSnapshotFeature()) != null) {
166              lastSnapshot = sf.getLastSnapshotId();
167            }
168            snapshotId = lastSnapshot;
169          }
170        }
171      }
172      if (curNode.isSymlink() && (!lastComp || resolveLink)) {
173        final String path = constructPath(components, 0, components.length);
174        final String preceding = constructPath(components, 0, count);
175        final String remainder =
176          constructPath(components, count + 1, components.length);
177        final String link = DFSUtil.bytes2String(components[count]);
178        final String target = curNode.asSymlink().getSymlinkString();
179        if (LOG.isDebugEnabled()) {
180          LOG.debug("UnresolvedPathException " +
181            " path: " + path + " preceding: " + preceding +
182            " count: " + count + " link: " + link + " target: " + target +
183            " remainder: " + remainder);
184        }
185        throw new UnresolvedPathException(path, preceding, remainder, target);
186      }
187      if (lastComp || !isDir) {
188        break;
189      }
190      final byte[] childName = components[count + 1];
191      
192      // check if the next byte[] in components is for ".snapshot"
193      if (isDotSnapshotDir(childName) && dir.isSnapshottable()) {
194        // skip the ".snapshot" in components
195        count++;
196        isSnapshot = true;
197        // check if ".snapshot" is the last element of components
198        if (count == components.length - 1) {
199          break;
200        }
201        // Resolve snapshot root
202        final Snapshot s = dir.getSnapshot(components[count + 1]);
203        if (s == null) {
204          curNode = null; // snapshot not found
205        } else {
206          curNode = s.getRoot();
207          snapshotId = s.getId();
208        }
209      } else {
210        // normal case, and also for resolving file/dir under snapshot root
211        curNode = dir.getChild(childName,
212            isSnapshot ? snapshotId : CURRENT_STATE_ID);
213      }
214      count++;
215    }
216    if (isSnapshot && !isDotSnapshotDir(components[components.length - 1])) {
217      // for snapshot path shrink the inode array. however, for path ending with
218      // .snapshot, still keep last the null inode in the array
219      INode[] newNodes = new INode[components.length - 1];
220      System.arraycopy(inodes, 0, newNodes, 0, newNodes.length);
221      inodes = newNodes;
222    }
223    return new INodesInPath(inodes, components, isSnapshot, snapshotId);
224  }
225
226  private static boolean shouldUpdateLatestId(int sid, int snapshotId) {
227    return snapshotId == CURRENT_STATE_ID || (sid != CURRENT_STATE_ID &&
228        ID_INTEGER_COMPARATOR.compare(snapshotId, sid) < 0);
229  }
230
231  /**
232   * Replace an inode of the given INodesInPath in the given position. We do a
233   * deep copy of the INode array.
234   * @param pos the position of the replacement
235   * @param inode the new inode
236   * @return a new INodesInPath instance
237   */
238  public static INodesInPath replace(INodesInPath iip, int pos, INode inode) {
239    Preconditions.checkArgument(iip.length() > 0 && pos > 0 // no for root
240        && pos < iip.length());
241    if (iip.getINode(pos) == null) {
242      Preconditions.checkState(iip.getINode(pos - 1) != null);
243    }
244    INode[] inodes = new INode[iip.inodes.length];
245    System.arraycopy(iip.inodes, 0, inodes, 0, inodes.length);
246    inodes[pos] = inode;
247    return new INodesInPath(inodes, iip.path, iip.isSnapshot, iip.snapshotId);
248  }
249
250  /**
251   * Extend a given INodesInPath with a child INode. The child INode will be
252   * appended to the end of the new INodesInPath.
253   */
254  public static INodesInPath append(INodesInPath iip, INode child,
255      byte[] childName) {
256    Preconditions.checkArgument(!iip.isSnapshot && iip.length() > 0);
257    Preconditions.checkArgument(iip.getLastINode() != null && iip
258        .getLastINode().isDirectory());
259    INode[] inodes = new INode[iip.length() + 1];
260    System.arraycopy(iip.inodes, 0, inodes, 0, inodes.length - 1);
261    inodes[inodes.length - 1] = child;
262    byte[][] path = new byte[iip.path.length + 1][];
263    System.arraycopy(iip.path, 0, path, 0, path.length - 1);
264    path[path.length - 1] = childName;
265    return new INodesInPath(inodes, path, false, iip.snapshotId);
266  }
267
268  private final byte[][] path;
269  /**
270   * Array with the specified number of INodes resolved for a given path.
271   */
272  private final INode[] inodes;
273  /**
274   * true if this path corresponds to a snapshot
275   */
276  private final boolean isSnapshot;
277  /**
278   * For snapshot paths, it is the id of the snapshot; or 
279   * {@link Snapshot#CURRENT_STATE_ID} if the snapshot does not exist. For 
280   * non-snapshot paths, it is the id of the latest snapshot found in the path;
281   * or {@link Snapshot#CURRENT_STATE_ID} if no snapshot is found.
282   */
283  private final int snapshotId;
284
285  private INodesInPath(INode[] inodes, byte[][] path, boolean isSnapshot,
286      int snapshotId) {
287    Preconditions.checkArgument(inodes != null && path != null);
288    this.inodes = inodes;
289    this.path = path;
290    this.isSnapshot = isSnapshot;
291    this.snapshotId = snapshotId;
292  }
293
294  private INodesInPath(INode[] inodes, byte[][] path) {
295    this(inodes, path, false, CURRENT_STATE_ID);
296  }
297
298  /**
299   * For non-snapshot paths, return the latest snapshot id found in the path.
300   */
301  public int getLatestSnapshotId() {
302    Preconditions.checkState(!isSnapshot);
303    return snapshotId;
304  }
305  
306  /**
307   * For snapshot paths, return the id of the snapshot specified in the path.
308   * For non-snapshot paths, return {@link Snapshot#CURRENT_STATE_ID}.
309   */
310  public int getPathSnapshotId() {
311    return isSnapshot ? snapshotId : CURRENT_STATE_ID;
312  }
313
314  /**
315   * @return the i-th inode if i >= 0;
316   *         otherwise, i < 0, return the (length + i)-th inode.
317   */
318  public INode getINode(int i) {
319    if (inodes == null || inodes.length == 0) {
320      throw new NoSuchElementException("inodes is null or empty");
321    }
322    int index = i >= 0 ? i : inodes.length + i;
323    if (index < inodes.length && index >= 0) {
324      return inodes[index];
325    } else {
326      throw new NoSuchElementException("inodes.length == " + inodes.length);
327    }
328  }
329  
330  /** @return the last inode. */
331  public INode getLastINode() {
332    return getINode(-1);
333  }
334
335  byte[] getLastLocalName() {
336    return path[path.length - 1];
337  }
338
339  public byte[][] getPathComponents() {
340    return path;
341  }
342
343  /** @return the full path in string form */
344  public String getPath() {
345    return DFSUtil.byteArray2PathString(path);
346  }
347
348  public String getParentPath() {
349    return getPath(path.length - 1);
350  }
351
352  public String getPath(int pos) {
353    return DFSUtil.byteArray2PathString(path, 0, pos);
354  }
355
356  /**
357   * @param offset start endpoint (inclusive)
358   * @param length number of path components
359   * @return sub-list of the path
360   */
361  public List<String> getPath(int offset, int length) {
362    Preconditions.checkArgument(offset >= 0 && length >= 0 && offset + length
363        <= path.length);
364    ImmutableList.Builder<String> components = ImmutableList.builder();
365    for (int i = offset; i < offset + length; i++) {
366      components.add(DFSUtil.bytes2String(path[i]));
367    }
368    return components.build();
369  }
370
371  public int length() {
372    return inodes.length;
373  }
374
375  public List<INode> getReadOnlyINodes() {
376    return Collections.unmodifiableList(Arrays.asList(inodes));
377  }
378
379  public INode[] getINodesArray() {
380    INode[] retArr = new INode[inodes.length];
381    System.arraycopy(inodes, 0, retArr, 0, inodes.length);
382    return retArr;
383  }
384
385  /**
386   * @param length number of ancestral INodes in the returned INodesInPath
387   *               instance
388   * @return the INodesInPath instance containing ancestral INodes. Note that
389   * this method only handles non-snapshot paths.
390   */
391  private INodesInPath getAncestorINodesInPath(int length) {
392    Preconditions.checkArgument(length >= 0 && length < inodes.length);
393    Preconditions.checkState(!isSnapshot());
394    final INode[] anodes = new INode[length];
395    final byte[][] apath = new byte[length][];
396    System.arraycopy(this.inodes, 0, anodes, 0, length);
397    System.arraycopy(this.path, 0, apath, 0, length);
398    return new INodesInPath(anodes, apath, false, snapshotId);
399  }
400
401  /**
402   * @return an INodesInPath instance containing all the INodes in the parent
403   *         path. We do a deep copy here.
404   */
405  public INodesInPath getParentINodesInPath() {
406    return inodes.length > 1 ? getAncestorINodesInPath(inodes.length - 1) :
407        null;
408  }
409
410  /**
411   * @return a new INodesInPath instance that only contains exisitng INodes.
412   * Note that this method only handles non-snapshot paths.
413   */
414  public INodesInPath getExistingINodes() {
415    Preconditions.checkState(!isSnapshot());
416    int i = 0;
417    for (; i < inodes.length; i++) {
418      if (inodes[i] == null) {
419        break;
420      }
421    }
422    INode[] existing = new INode[i];
423    byte[][] existingPath = new byte[i][];
424    System.arraycopy(inodes, 0, existing, 0, i);
425    System.arraycopy(path, 0, existingPath, 0, i);
426    return new INodesInPath(existing, existingPath, false, snapshotId);
427  }
428
429  /**
430   * @return isSnapshot true for a snapshot path
431   */
432  boolean isSnapshot() {
433    return this.isSnapshot;
434  }
435
436  private static String toString(INode inode) {
437    return inode == null? null: inode.getLocalName();
438  }
439
440  @Override
441  public String toString() {
442    return toString(true);
443  }
444
445  private String toString(boolean vaildateObject) {
446    if (vaildateObject) {
447      validate();
448    }
449
450    final StringBuilder b = new StringBuilder(getClass().getSimpleName())
451        .append(": path = ").append(DFSUtil.byteArray2PathString(path))
452        .append("\n  inodes = ");
453    if (inodes == null) {
454      b.append("null");
455    } else if (inodes.length == 0) {
456      b.append("[]");
457    } else {
458      b.append("[").append(toString(inodes[0]));
459      for(int i = 1; i < inodes.length; i++) {
460        b.append(", ").append(toString(inodes[i]));
461      }
462      b.append("], length=").append(inodes.length);
463    }
464    b.append("\n  isSnapshot        = ").append(isSnapshot)
465     .append("\n  snapshotId        = ").append(snapshotId);
466    return b.toString();
467  }
468
469  void validate() {
470    // check parent up to snapshotRootIndex if this is a snapshot path
471    int i = 0;
472    if (inodes[i] != null) {
473      for(i++; i < inodes.length && inodes[i] != null; i++) {
474        final INodeDirectory parent_i = inodes[i].getParent();
475        final INodeDirectory parent_i_1 = inodes[i-1].getParent();
476        if (parent_i != inodes[i-1] &&
477            (parent_i_1 == null || !parent_i_1.isSnapshottable()
478                || parent_i != parent_i_1)) {
479          throw new AssertionError(
480              "inodes[" + i + "].getParent() != inodes[" + (i-1)
481              + "]\n  inodes[" + i + "]=" + inodes[i].toDetailString()
482              + "\n  inodes[" + (i-1) + "]=" + inodes[i-1].toDetailString()
483              + "\n this=" + toString(false));
484        }
485      }
486    }
487    if (i != inodes.length) {
488      throw new AssertionError("i = " + i + " != " + inodes.length
489          + ", this=" + toString(false));
490    }
491  }
492}