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.io.IOException;
021import java.util.List;
022
023import org.apache.hadoop.fs.permission.FsAction;
024import org.apache.hadoop.fs.permission.FsPermission;
025import org.apache.hadoop.fs.permission.PermissionStatus;
026import org.apache.hadoop.hdfs.protocol.Block;
027import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
028import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
029
030/** I-node for closed file. */
031public class INodeFile extends INode {
032  static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
033
034  //Number of bits for Block size
035  static final short BLOCKBITS = 48;
036
037  //Header mask 64-bit representation
038  //Format: [16 bits for replication][48 bits for PreferredBlockSize]
039  static final long HEADERMASK = 0xffffL << BLOCKBITS;
040
041  protected long header;
042
043  protected BlockInfo blocks[] = null;
044
045  INodeFile(PermissionStatus permissions,
046            int nrBlocks, short replication, long modificationTime,
047            long atime, long preferredBlockSize) {
048    this(permissions, new BlockInfo[nrBlocks], replication,
049        modificationTime, atime, preferredBlockSize);
050  }
051
052  protected INodeFile() {
053    blocks = null;
054    header = 0;
055  }
056
057  protected INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
058                      short replication, long modificationTime,
059                      long atime, long preferredBlockSize) {
060    super(permissions, modificationTime, atime);
061    this.setReplication(replication);
062    this.setPreferredBlockSize(preferredBlockSize);
063    blocks = blklist;
064  }
065
066  /**
067   * Set the {@link FsPermission} of this {@link INodeFile}.
068   * Since this is a file,
069   * the {@link FsAction#EXECUTE} action, if any, is ignored.
070   */
071  protected void setPermission(FsPermission permission) {
072    super.setPermission(permission.applyUMask(UMASK));
073  }
074
075  public boolean isDirectory() {
076    return false;
077  }
078
079  /**
080   * Get block replication for the file 
081   * @return block replication value
082   */
083  public short getReplication() {
084    return (short) ((header & HEADERMASK) >> BLOCKBITS);
085  }
086
087  public void setReplication(short replication) {
088    if(replication <= 0)
089       throw new IllegalArgumentException("Unexpected value for the replication");
090    header = ((long)replication << BLOCKBITS) | (header & ~HEADERMASK);
091  }
092
093  /**
094   * Get preferred block size for the file
095   * @return preferred block size in bytes
096   */
097  public long getPreferredBlockSize() {
098        return header & ~HEADERMASK;
099  }
100
101  public void setPreferredBlockSize(long preferredBlkSize)
102  {
103    if((preferredBlkSize < 0) || (preferredBlkSize > ~HEADERMASK ))
104       throw new IllegalArgumentException("Unexpected value for the block size");
105    header = (header & HEADERMASK) | (preferredBlkSize & ~HEADERMASK);
106  }
107
108  /**
109   * Get file blocks 
110   * @return file blocks
111   */
112  public BlockInfo[] getBlocks() {
113    return this.blocks;
114  }
115
116  /**
117   * append array of blocks to this.blocks
118   */
119  void appendBlocks(INodeFile [] inodes, int totalAddedBlocks) {
120    int size = this.blocks.length;
121    
122    BlockInfo[] newlist = new BlockInfo[size + totalAddedBlocks];
123    System.arraycopy(this.blocks, 0, newlist, 0, size);
124    
125    for(INodeFile in: inodes) {
126      System.arraycopy(in.blocks, 0, newlist, size, in.blocks.length);
127      size += in.blocks.length;
128    }
129    
130    for(BlockInfo bi: newlist) {
131      bi.setINode(this);
132    }
133    this.blocks = newlist;
134  }
135  
136  /**
137   * add a block to the block list
138   */
139  void addBlock(BlockInfo newblock) {
140    if (this.blocks == null) {
141      this.blocks = new BlockInfo[1];
142      this.blocks[0] = newblock;
143    } else {
144      int size = this.blocks.length;
145      BlockInfo[] newlist = new BlockInfo[size + 1];
146      System.arraycopy(this.blocks, 0, newlist, 0, size);
147      newlist[size] = newblock;
148      this.blocks = newlist;
149    }
150  }
151
152  /**
153   * Set file block
154   */
155  public void setBlock(int idx, BlockInfo blk) {
156    this.blocks[idx] = blk;
157  }
158
159  int collectSubtreeBlocksAndClear(List<Block> v) {
160    parent = null;
161    if(blocks != null && v != null) {
162      for (BlockInfo blk : blocks) {
163        v.add(blk);
164        blk.setINode(null);
165      }
166    }
167    blocks = null;
168    return 1;
169  }
170
171  /** {@inheritDoc} */
172  ContentSummaryComputationContext computeContentSummary(
173     ContentSummaryComputationContext summary) {
174    summary.updateCounts(computeFileSize(true), 1, 0,
175        diskspaceConsumed());
176    return summary;
177  }
178
179  /** Compute file size.
180   * May or may not include BlockInfoUnderConstruction.
181   */
182  long computeFileSize(boolean includesBlockInfoUnderConstruction) {
183    if (blocks == null || blocks.length == 0) {
184      return 0;
185    }
186    final int last = blocks.length - 1;
187    //check if the last block is BlockInfoUnderConstruction
188    long bytes = blocks[last] instanceof BlockInfoUnderConstruction
189                 && !includesBlockInfoUnderConstruction?
190                     0: blocks[last].getNumBytes();
191    for(int i = 0; i < last; i++) {
192      bytes += blocks[i].getNumBytes();
193    }
194    return bytes;
195  }
196  
197
198  @Override
199  DirCounts spaceConsumedInTree(DirCounts counts) {
200    counts.nsCount += 1;
201    counts.dsCount += diskspaceConsumed();
202    return counts;
203  }
204
205  long diskspaceConsumed() {
206    return diskspaceConsumed(blocks);
207  }
208  
209  long diskspaceConsumed(Block[] blkArr) {
210    long size = 0;
211    if(blkArr == null || blocks.length == 0) { 
212      return 0;
213    }
214    
215    for (Block blk : blkArr) {
216      if (blk != null) {
217        size += blk.getNumBytes();
218      }
219    }
220    /* If the last block is being written to, use prefferedBlockSize
221     * rather than the actual block size.
222     */
223    final int last = blkArr.length - 1;
224    if (blkArr.length > 0 && blkArr[last] != null && 
225        blkArr[last] instanceof BlockInfoUnderConstruction) {
226      size += getPreferredBlockSize() - blkArr[last].getNumBytes();
227    }
228    return size * getReplication();
229  }
230  
231  /**
232   * Return the penultimate allocated block for this file.
233   */
234  BlockInfo getPenultimateBlock() {
235    if (blocks == null || blocks.length <= 1) {
236      return null;
237    }
238    return blocks[blocks.length - 2];
239  }
240
241  /**
242   * Get the last block of the file.
243   * Make sure it has the right type.
244   */
245  public <T extends BlockInfo> T getLastBlock() throws IOException {
246    if (blocks == null || blocks.length == 0)
247      return null;
248    T returnBlock = null;
249    try {
250      @SuppressWarnings("unchecked")  // ClassCastException is caught below
251      T tBlock = (T)blocks[blocks.length - 1];
252      returnBlock = tBlock;
253    } catch(ClassCastException cce) {
254      throw new IOException("Unexpected last block type: " 
255          + blocks[blocks.length - 1].getClass().getSimpleName());
256    }
257    return returnBlock;
258  }
259
260  /** @return the number of blocks */ 
261  public int numBlocks() {
262    return blocks == null ? 0 : blocks.length;
263  }
264}