source: proiecte/HadoopJUnit/hadoop-0.20.1/src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 10.7 KB
Line 
1/**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements.  See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership.  The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License.  You may obtain a copy of the License at
9 *
10 *     http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18package org.apache.hadoop.hdfs.server.namenode;
19
20import java.io.FileNotFoundException;
21import java.util.ArrayList;
22import java.util.Collections;
23import java.util.List;
24
25import org.apache.hadoop.fs.permission.FsAction;
26import org.apache.hadoop.fs.permission.FsPermission;
27import org.apache.hadoop.fs.permission.PermissionStatus;
28import org.apache.hadoop.hdfs.protocol.Block;
29
30/**
31 * Directory INode class.
32 */
33class INodeDirectory extends INode {
34  protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
35  final static String ROOT_NAME = "";
36
37  private List<INode> children;
38
39  INodeDirectory(String name, PermissionStatus permissions) {
40    super(name, permissions);
41    this.children = null;
42  }
43
44  public INodeDirectory(PermissionStatus permissions, long mTime) {
45    super(permissions, mTime, 0);
46    this.children = null;
47  }
48
49  /** constructor */
50  INodeDirectory(byte[] localName, PermissionStatus permissions, long mTime) {
51    this(permissions, mTime);
52    this.name = localName;
53  }
54 
55  /** copy constructor
56   *
57   * @param other
58   */
59  INodeDirectory(INodeDirectory other) {
60    super(other);
61    this.children = other.getChildren();
62  }
63 
64  /**
65   * Check whether it's a directory
66   */
67  public boolean isDirectory() {
68    return true;
69  }
70
71  INode removeChild(INode node) {
72    assert children != null;
73    int low = Collections.binarySearch(children, node.name);
74    if (low >= 0) {
75      return children.remove(low);
76    } else {
77      return null;
78    }
79  }
80
81  /** Replace a child that has the same name as newChild by newChild.
82   *
83   * @param newChild Child node to be added
84   */
85  void replaceChild(INode newChild) {
86    if ( children == null ) {
87      throw new IllegalArgumentException("The directory is empty");
88    }
89    int low = Collections.binarySearch(children, newChild.name);
90    if (low>=0) { // an old child exists so replace by the newChild
91      children.set(low, newChild);
92    } else {
93      throw new IllegalArgumentException("No child exists to be replaced");
94    }
95  }
96 
97  INode getChild(String name) {
98    return getChildINode(string2Bytes(name));
99  }
100
101  private INode getChildINode(byte[] name) {
102    if (children == null) {
103      return null;
104    }
105    int low = Collections.binarySearch(children, name);
106    if (low >= 0) {
107      return children.get(low);
108    }
109    return null;
110  }
111
112  /**
113   */
114  private INode getNode(byte[][] components) {
115    INode[] inode  = new INode[1];
116    getExistingPathINodes(components, inode);
117    return inode[0];
118  }
119
120  /**
121   * This is the external interface
122   */
123  INode getNode(String path) {
124    return getNode(getPathComponents(path));
125  }
126
127  /**
128   * Retrieve existing INodes from a path. If existing is big enough to store
129   * all path components (existing and non-existing), then existing INodes
130   * will be stored starting from the root INode into existing[0]; if
131   * existing is not big enough to store all path components, then only the
132   * last existing and non existing INodes will be stored so that
133   * existing[existing.length-1] refers to the target INode.
134   *
135   * <p>
136   * Example: <br>
137   * Given the path /c1/c2/c3 where only /c1/c2 exists, resulting in the
138   * following path components: ["","c1","c2","c3"],
139   *
140   * <p>
141   * <code>getExistingPathINodes(["","c1","c2"], [?])</code> should fill the
142   * array with [c2] <br>
143   * <code>getExistingPathINodes(["","c1","c2","c3"], [?])</code> should fill the
144   * array with [null]
145   *
146   * <p>
147   * <code>getExistingPathINodes(["","c1","c2"], [?,?])</code> should fill the
148   * array with [c1,c2] <br>
149   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?])</code> should fill
150   * the array with [c2,null]
151   *
152   * <p>
153   * <code>getExistingPathINodes(["","c1","c2"], [?,?,?,?])</code> should fill
154   * the array with [rootINode,c1,c2,null], <br>
155   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?,?,?])</code> should
156   * fill the array with [rootINode,c1,c2,null]
157   * @param components array of path component name
158   * @param existing INode array to fill with existing INodes
159   * @return number of existing INodes in the path
160   */
161  int getExistingPathINodes(byte[][] components, INode[] existing) {
162    assert compareBytes(this.name, components[0]) == 0 :
163      "Incorrect name " + getLocalName() + " expected " + components[0];
164
165    INode curNode = this;
166    int count = 0;
167    int index = existing.length - components.length;
168    if (index > 0)
169      index = 0;
170    while ((count < components.length) && (curNode != null)) {
171      if (index >= 0)
172        existing[index] = curNode;
173      if (!curNode.isDirectory() || (count == components.length - 1))
174        break; // no more child, stop here
175      INodeDirectory parentDir = (INodeDirectory)curNode;
176      curNode = parentDir.getChildINode(components[count + 1]);
177      count += 1;
178      index += 1;
179    }
180    return count;
181  }
182
183  /**
184   * Retrieve the existing INodes along the given path. The first INode
185   * always exist and is this INode.
186   *
187   * @param path the path to explore
188   * @return INodes array containing the existing INodes in the order they
189   *         appear when following the path from the root INode to the
190   *         deepest INodes. The array size will be the number of expected
191   *         components in the path, and non existing components will be
192   *         filled with null
193   *         
194   * @see #getExistingPathINodes(byte[][], INode[])
195   */
196  INode[] getExistingPathINodes(String path) {
197    byte[][] components = getPathComponents(path);
198    INode[] inodes = new INode[components.length];
199
200    this.getExistingPathINodes(components, inodes);
201   
202    return inodes;
203  }
204
205  /**
206   * Add a child inode to the directory.
207   *
208   * @param node INode to insert
209   * @param inheritPermission inherit permission from parent?
210   * @return  null if the child with this name already exists;
211   *          node, otherwise
212   */
213  <T extends INode> T addChild(final T node, boolean inheritPermission) {
214    if (inheritPermission) {
215      FsPermission p = getFsPermission();
216      //make sure the  permission has wx for the user
217      if (!p.getUserAction().implies(FsAction.WRITE_EXECUTE)) {
218        p = new FsPermission(p.getUserAction().or(FsAction.WRITE_EXECUTE),
219            p.getGroupAction(), p.getOtherAction());
220      }
221      node.setPermission(p);
222    }
223
224    if (children == null) {
225      children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
226    }
227    int low = Collections.binarySearch(children, node.name);
228    if(low >= 0)
229      return null;
230    node.parent = this;
231    children.add(-low - 1, node);
232    // update modification time of the parent directory
233    setModificationTime(node.getModificationTime());
234    if (node.getGroupName() == null) {
235      node.setGroup(getGroupName());
236    }
237    return node;
238  }
239
240  /**
241   * Equivalent to addNode(path, newNode, false).
242   * @see #addNode(String, INode, boolean)
243   */
244  <T extends INode> T addNode(String path, T newNode) throws FileNotFoundException {
245    return addNode(path, newNode, false);
246  }
247  /**
248   * Add new INode to the file tree.
249   * Find the parent and insert
250   *
251   * @param path file path
252   * @param newNode INode to be added
253   * @param inheritPermission If true, copy the parent's permission to newNode.
254   * @return null if the node already exists; inserted INode, otherwise
255   * @throws FileNotFoundException if parent does not exist or
256   * is not a directory.
257   */
258  <T extends INode> T addNode(String path, T newNode, boolean inheritPermission
259      ) throws FileNotFoundException {
260    if(addToParent(path, newNode, null, inheritPermission) == null)
261      return null;
262    return newNode;
263  }
264
265  /**
266   * Add new inode to the parent if specified.
267   * Optimized version of addNode() if parent is not null.
268   *
269   * @return  parent INode if new inode is inserted
270   *          or null if it already exists.
271   * @throws  FileNotFoundException if parent does not exist or
272   *          is not a directory.
273   */
274  <T extends INode> INodeDirectory addToParent(
275                                      String path,
276                                      T newNode,
277                                      INodeDirectory parent,
278                                      boolean inheritPermission
279                                    ) throws FileNotFoundException {
280    byte[][] pathComponents = getPathComponents(path);
281    assert pathComponents != null : "Incorrect path " + path;
282    int pathLen = pathComponents.length;
283    if (pathLen < 2)  // add root
284      return null;
285    if(parent == null) {
286      // Gets the parent INode
287      INode[] inodes  = new INode[2];
288      getExistingPathINodes(pathComponents, inodes);
289      INode inode = inodes[0];
290      if (inode == null) {
291        throw new FileNotFoundException("Parent path does not exist: "+path);
292      }
293      if (!inode.isDirectory()) {
294        throw new FileNotFoundException("Parent path is not a directory: "+path);
295      }
296      parent = (INodeDirectory)inode;
297    }
298    // insert into the parent children list
299    newNode.name = pathComponents[pathLen-1];
300    if(parent.addChild(newNode, inheritPermission) == null)
301      return null;
302    return parent;
303  }
304
305  /** {@inheritDoc} */
306  DirCounts spaceConsumedInTree(DirCounts counts) {
307    counts.nsCount += 1;
308    if (children != null) {
309      for (INode child : children) {
310        child.spaceConsumedInTree(counts);
311      }
312    }
313    return counts;   
314  }
315
316  /** {@inheritDoc} */
317  long[] computeContentSummary(long[] summary) {
318    if (children != null) {
319      for (INode child : children) {
320        child.computeContentSummary(summary);
321      }
322    }
323    summary[2]++;
324    return summary;
325  }
326
327  /**
328   */
329  List<INode> getChildren() {
330    return children==null ? new ArrayList<INode>() : children;
331  }
332  List<INode> getChildrenRaw() {
333    return children;
334  }
335
336  int collectSubtreeBlocksAndClear(List<Block> v) {
337    int total = 1;
338    if (children == null) {
339      return total;
340    }
341    for (INode child : children) {
342      total += child.collectSubtreeBlocksAndClear(v);
343    }
344    parent = null;
345    children = null;
346    return total;
347  }
348}
Note: See TracBrowser for help on using the repository browser.