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.util.Arrays;
021    import java.util.List;
022    
023    import org.apache.hadoop.fs.ContentSummary;
024    import org.apache.hadoop.fs.Path;
025    import org.apache.hadoop.fs.permission.FsPermission;
026    import org.apache.hadoop.fs.permission.PermissionStatus;
027    import org.apache.hadoop.hdfs.DFSUtil;
028    import org.apache.hadoop.hdfs.protocol.Block;
029    import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
030    import org.apache.hadoop.util.StringUtils;
031    
032    /**
033     * We keep an in-memory representation of the file/block hierarchy.
034     * This is a base INode class containing common fields for file and 
035     * directory inodes.
036     */
037    public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
038      /*
039       *  The inode name is in java UTF8 encoding; 
040       *  The name in HdfsFileStatus should keep the same encoding as this.
041       *  if this encoding is changed, implicitly getFileInfo and listStatus in
042       *  clientProtocol are changed; The decoding at the client
043       *  side should change accordingly.
044       */
045      protected byte[] name;
046      protected INodeDirectory parent;
047      protected long modificationTime;
048      protected long accessTime;
049    
050      /** Simple wrapper for two counters : 
051       *  nsCount (namespace consumed) and dsCount (diskspace consumed).
052       */
053      static class DirCounts {
054        long nsCount = 0;
055        long dsCount = 0;
056        
057        /** returns namespace count */
058        long getNsCount() {
059          return nsCount;
060        }
061        /** returns diskspace count */
062        long getDsCount() {
063          return dsCount;
064        }
065      }
066      
067      //Only updated by updatePermissionStatus(...).
068      //Other codes should not modify it.
069      private long permission;
070    
071      private static enum PermissionStatusFormat {
072        MODE(0, 16),
073        GROUP(MODE.OFFSET + MODE.LENGTH, 25),
074        USER(GROUP.OFFSET + GROUP.LENGTH, 23);
075    
076        final int OFFSET;
077        final int LENGTH; //bit length
078        final long MASK;
079    
080        PermissionStatusFormat(int offset, int length) {
081          OFFSET = offset;
082          LENGTH = length;
083          MASK = ((-1L) >>> (64 - LENGTH)) << OFFSET;
084        }
085    
086        long retrieve(long record) {
087          return (record & MASK) >>> OFFSET;
088        }
089    
090        long combine(long bits, long record) {
091          return (record & ~MASK) | (bits << OFFSET);
092        }
093      }
094    
095      protected INode() {
096        name = null;
097        parent = null;
098        modificationTime = 0;
099        accessTime = 0;
100      }
101    
102      INode(PermissionStatus permissions, long mTime, long atime) {
103        this.name = null;
104        this.parent = null;
105        this.modificationTime = mTime;
106        setAccessTime(atime);
107        setPermissionStatus(permissions);
108      }
109    
110      protected INode(String name, PermissionStatus permissions) {
111        this(permissions, 0L, 0L);
112        setLocalName(name);
113      }
114      
115      /** copy constructor
116       * 
117       * @param other Other node to be copied
118       */
119      INode(INode other) {
120        setLocalName(other.getLocalName());
121        this.parent = other.getParent();
122        setPermissionStatus(other.getPermissionStatus());
123        setModificationTime(other.getModificationTime());
124        setAccessTime(other.getAccessTime());
125      }
126    
127      /**
128       * Check whether this is the root inode.
129       */
130      boolean isRoot() {
131        return name.length == 0;
132      }
133    
134      /** Set the {@link PermissionStatus} */
135      protected void setPermissionStatus(PermissionStatus ps) {
136        setUser(ps.getUserName());
137        setGroup(ps.getGroupName());
138        setPermission(ps.getPermission());
139      }
140      /** Get the {@link PermissionStatus} */
141      protected PermissionStatus getPermissionStatus() {
142        return new PermissionStatus(getUserName(),getGroupName(),getFsPermission());
143      }
144      private synchronized void updatePermissionStatus(
145          PermissionStatusFormat f, long n) {
146        permission = f.combine(n, permission);
147      }
148      /** Get user name */
149      public String getUserName() {
150        int n = (int)PermissionStatusFormat.USER.retrieve(permission);
151        return SerialNumberManager.INSTANCE.getUser(n);
152      }
153      /** Set user */
154      protected void setUser(String user) {
155        int n = SerialNumberManager.INSTANCE.getUserSerialNumber(user);
156        updatePermissionStatus(PermissionStatusFormat.USER, n);
157      }
158      /** Get group name */
159      public String getGroupName() {
160        int n = (int)PermissionStatusFormat.GROUP.retrieve(permission);
161        return SerialNumberManager.INSTANCE.getGroup(n);
162      }
163      /** Set group */
164      protected void setGroup(String group) {
165        int n = SerialNumberManager.INSTANCE.getGroupSerialNumber(group);
166        updatePermissionStatus(PermissionStatusFormat.GROUP, n);
167      }
168      /** Get the {@link FsPermission} */
169      public FsPermission getFsPermission() {
170        return new FsPermission(
171            (short)PermissionStatusFormat.MODE.retrieve(permission));
172      }
173      protected short getFsPermissionShort() {
174        return (short)PermissionStatusFormat.MODE.retrieve(permission);
175      }
176      /** Set the {@link FsPermission} of this {@link INode} */
177      protected void setPermission(FsPermission permission) {
178        updatePermissionStatus(PermissionStatusFormat.MODE, permission.toShort());
179      }
180    
181      /**
182       * Check whether it's a directory
183       */
184      public abstract boolean isDirectory();
185    
186      /**
187       * Collect all the blocks in all children of this INode.
188       * Count and return the number of files in the sub tree.
189       * Also clears references since this INode is deleted.
190       */
191      abstract int collectSubtreeBlocksAndClear(List<Block> v);
192    
193      /** Compute {@link ContentSummary}. */
194      public final ContentSummary computeContentSummary() {
195        long[] a = computeContentSummary(new long[]{0,0,0,0});
196        return new ContentSummary(a[0], a[1], a[2], getNsQuota(), 
197                                  a[3], getDsQuota());
198      }
199      /**
200       * @return an array of three longs. 
201       * 0: length, 1: file count, 2: directory count 3: disk space
202       */
203      abstract long[] computeContentSummary(long[] summary);
204      
205      /**
206       * Get the quota set for this inode
207       * @return the quota if it is set; -1 otherwise
208       */
209      long getNsQuota() {
210        return -1;
211      }
212    
213      long getDsQuota() {
214        return -1;
215      }
216      
217      boolean isQuotaSet() {
218        return getNsQuota() >= 0 || getDsQuota() >= 0;
219      }
220      
221      /**
222       * Adds total number of names and total disk space taken under 
223       * this tree to counts.
224       * Returns updated counts object.
225       */
226      abstract DirCounts spaceConsumedInTree(DirCounts counts);
227      
228      /**
229       * Get local file name
230       * @return local file name
231       */
232      String getLocalName() {
233        return DFSUtil.bytes2String(name);
234      }
235    
236    
237      String getLocalParentDir() {
238        INode inode = isRoot() ? this : getParent();
239        return (inode != null) ? inode.getFullPathName() : "";
240      }
241    
242      /**
243       * Get local file name
244       * @return local file name
245       */
246      byte[] getLocalNameBytes() {
247        return name;
248      }
249    
250      /**
251       * Set local file name
252       */
253      void setLocalName(String name) {
254        this.name = DFSUtil.string2Bytes(name);
255      }
256    
257      /**
258       * Set local file name
259       */
260      void setLocalName(byte[] name) {
261        this.name = name;
262      }
263    
264      /** {@inheritDoc} */
265      public String getFullPathName() {
266        // Get the full path name of this inode.
267        return FSDirectory.getFullPathName(this);
268      }
269    
270      /** {@inheritDoc} */
271      public String toString() {
272        return "\"" + getFullPathName() + "\":"
273        + getUserName() + ":" + getGroupName() + ":"
274        + (isDirectory()? "d": "-") + getFsPermission();
275      }
276    
277      /**
278       * Get parent directory 
279       * @return parent INode
280       */
281      INodeDirectory getParent() {
282        return this.parent;
283      }
284    
285      /** 
286       * Get last modification time of inode.
287       * @return access time
288       */
289      public long getModificationTime() {
290        return this.modificationTime;
291      }
292    
293      /**
294       * Set last modification time of inode.
295       */
296      void setModificationTime(long modtime) {
297        assert isDirectory();
298        if (this.modificationTime <= modtime) {
299          this.modificationTime = modtime;
300        }
301      }
302    
303      /**
304       * Always set the last modification time of inode.
305       */
306      void setModificationTimeForce(long modtime) {
307        this.modificationTime = modtime;
308      }
309    
310      /**
311       * Get access time of inode.
312       * @return access time
313       */
314      public long getAccessTime() {
315        return accessTime;
316      }
317    
318      /**
319       * Set last access time of inode.
320       */
321      void setAccessTime(long atime) {
322        accessTime = atime;
323      }
324    
325      /**
326       * Is this inode being constructed?
327       */
328      public boolean isUnderConstruction() {
329        return false;
330      }
331    
332      /**
333       * Check whether it's a symlink
334       */
335      public boolean isLink() {
336        return false;
337      }
338    
339      /**
340       * Breaks file path into components.
341       * @param path
342       * @return array of byte arrays each of which represents 
343       * a single path component.
344       */
345      static byte[][] getPathComponents(String path) {
346        return getPathComponents(getPathNames(path));
347      }
348    
349      /** Convert strings to byte arrays for path components. */
350      static byte[][] getPathComponents(String[] strings) {
351        if (strings.length == 0) {
352          return new byte[][]{null};
353        }
354        byte[][] bytes = new byte[strings.length][];
355        for (int i = 0; i < strings.length; i++)
356          bytes[i] = DFSUtil.string2Bytes(strings[i]);
357        return bytes;
358      }
359    
360      /**
361       * Splits an absolute path into an array of path components.
362       * @param path
363       * @throws AssertionError if the given path is invalid.
364       * @return array of path components.
365       */
366      static String[] getPathNames(String path) {
367        if (path == null || !path.startsWith(Path.SEPARATOR)) {
368          throw new AssertionError("Absolute path required");
369        }
370        return StringUtils.split(path, Path.SEPARATOR_CHAR);
371      }
372    
373      /**
374       * Given some components, create a path name.
375       * @param components The path components
376       * @param start index
377       * @param end index
378       * @return concatenated path
379       */
380      static String constructPath(byte[][] components, int start, int end) {
381        StringBuilder buf = new StringBuilder();
382        for (int i = start; i < end; i++) {
383          buf.append(DFSUtil.bytes2String(components[i]));
384          if (i < end - 1) {
385            buf.append(Path.SEPARATOR);
386          }
387        }
388        return buf.toString();
389      }
390    
391      boolean removeNode() {
392        if (parent == null) {
393          return false;
394        } else {
395          parent.removeChild(this);
396          parent = null;
397          return true;
398        }
399      }
400    
401      //
402      // Comparable interface
403      //
404      public int compareTo(byte[] o) {
405        return compareBytes(name, o);
406      }
407    
408      public boolean equals(Object o) {
409        if (!(o instanceof INode)) {
410          return false;
411        }
412        return Arrays.equals(this.name, ((INode)o).name);
413      }
414    
415      public int hashCode() {
416        return Arrays.hashCode(this.name);
417      }
418    
419      //
420      // static methods
421      //
422      /**
423       * Compare two byte arrays.
424       * 
425       * @return a negative integer, zero, or a positive integer 
426       * as defined by {@link #compareTo(byte[])}.
427       */
428      static int compareBytes(byte[] a1, byte[] a2) {
429        if (a1==a2)
430            return 0;
431        int len1 = (a1==null ? 0 : a1.length);
432        int len2 = (a2==null ? 0 : a2.length);
433        int n = Math.min(len1, len2);
434        byte b1, b2;
435        for (int i=0; i<n; i++) {
436          b1 = a1[i];
437          b2 = a2[i];
438          if (b1 != b2)
439            return b1 - b2;
440        }
441        return len1 - len2;
442      }
443      
444      /**
445       * Create an INode; the inode's name is not set yet
446       * 
447       * @param permissions permissions
448       * @param blocks blocks if a file
449       * @param symlink symblic link if a symbolic link
450       * @param replication replication factor
451       * @param modificationTime modification time
452       * @param atime access time
453       * @param nsQuota namespace quota
454       * @param dsQuota disk quota
455       * @param preferredBlockSize block size
456       * @return an inode
457       */
458      static INode newINode(PermissionStatus permissions,
459                            BlockInfo[] blocks,
460                            String symlink,
461                            short replication,
462                            long modificationTime,
463                            long atime,
464                            long nsQuota,
465                            long dsQuota,
466                            long preferredBlockSize) {
467        if (symlink.length() != 0) { // check if symbolic link
468          return new INodeSymlink(symlink, modificationTime, atime, permissions);
469        }  else if (blocks == null) { //not sym link and blocks null? directory!
470          if (nsQuota >= 0 || dsQuota >= 0) {
471            return new INodeDirectoryWithQuota(
472                permissions, modificationTime, nsQuota, dsQuota);
473          } 
474          // regular directory
475          return new INodeDirectory(permissions, modificationTime);
476        }
477        // file
478        return new INodeFile(permissions, blocks, replication,
479            modificationTime, atime, preferredBlockSize);
480      }
481    }