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.blockmanagement;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Iterator;
023import java.util.List;
024
025import org.apache.hadoop.hdfs.protocol.Block;
026import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
027import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
028import org.apache.hadoop.hdfs.server.namenode.NameNode;
029
030/**
031 * Represents a block that is currently being constructed.<br>
032 * This is usually the last block of a file opened for write or append.
033 */
034public class BlockInfoUnderConstruction extends BlockInfo {
035  /** Block state. See {@link BlockUCState} */
036  private BlockUCState blockUCState;
037
038  /**
039   * Block replicas as assigned when the block was allocated.
040   * This defines the pipeline order.
041   */
042  private List<ReplicaUnderConstruction> replicas;
043
044  /**
045   * Index of the primary data node doing the recovery. Useful for log
046   * messages.
047   */
048  private int primaryNodeIndex = -1;
049
050  /**
051   * The new generation stamp, which this block will have
052   * after the recovery succeeds. Also used as a recovery id to identify
053   * the right recovery if any of the abandoned recoveries re-appear.
054   */
055  private long blockRecoveryId = 0;
056
057  /**
058   * ReplicaUnderConstruction contains information about replicas while
059   * they are under construction.
060   * The GS, the length and the state of the replica is as reported by 
061   * the data-node.
062   * It is not guaranteed, but expected, that data-nodes actually have
063   * corresponding replicas.
064   */
065  static class ReplicaUnderConstruction extends Block {
066    private final DatanodeStorageInfo expectedLocation;
067    private ReplicaState state;
068    private boolean chosenAsPrimary;
069
070    ReplicaUnderConstruction(Block block,
071                             DatanodeStorageInfo target,
072                             ReplicaState state) {
073      super(block);
074      this.expectedLocation = target;
075      this.state = state;
076      this.chosenAsPrimary = false;
077    }
078
079    /**
080     * Expected block replica location as assigned when the block was allocated.
081     * This defines the pipeline order.
082     * It is not guaranteed, but expected, that the data-node actually has
083     * the replica.
084     */
085    private DatanodeStorageInfo getExpectedStorageLocation() {
086      return expectedLocation;
087    }
088
089    /**
090     * Get replica state as reported by the data-node.
091     */
092    ReplicaState getState() {
093      return state;
094    }
095
096    /**
097     * Whether the replica was chosen for recovery.
098     */
099    boolean getChosenAsPrimary() {
100      return chosenAsPrimary;
101    }
102
103    /**
104     * Set replica state.
105     */
106    void setState(ReplicaState s) {
107      state = s;
108    }
109
110    /**
111     * Set whether this replica was chosen for recovery.
112     */
113    void setChosenAsPrimary(boolean chosenAsPrimary) {
114      this.chosenAsPrimary = chosenAsPrimary;
115    }
116
117    /**
118     * Is data-node the replica belongs to alive.
119     */
120    boolean isAlive() {
121      return expectedLocation.getDatanodeDescriptor().isAlive;
122    }
123
124    @Override // Block
125    public int hashCode() {
126      return super.hashCode();
127    }
128
129    @Override // Block
130    public boolean equals(Object obj) {
131      // Sufficient to rely on super's implementation
132      return (this == obj) || super.equals(obj);
133    }
134
135    @Override
136    public String toString() {
137      final StringBuilder b = new StringBuilder(50);
138      appendStringTo(b);
139      return b.toString();
140    }
141    
142    @Override
143    public void appendStringTo(StringBuilder sb) {
144      sb.append("ReplicaUnderConstruction[")
145        .append(expectedLocation)
146        .append("|")
147        .append(state)
148        .append("]");
149    }
150  }
151
152  /**
153   * Create block and set its state to
154   * {@link BlockUCState#UNDER_CONSTRUCTION}.
155   */
156  public BlockInfoUnderConstruction(Block blk, int replication) {
157    this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
158  }
159
160  /**
161   * Create a block that is currently being constructed.
162   */
163  public BlockInfoUnderConstruction(Block blk, int replication,
164                             BlockUCState state,
165                             DatanodeStorageInfo[] targets) {
166    super(blk, replication);
167    assert getBlockUCState() != BlockUCState.COMPLETE :
168      "BlockInfoUnderConstruction cannot be in COMPLETE state";
169    this.blockUCState = state;
170    setExpectedLocations(targets);
171  }
172
173  /**
174   * Convert an under construction block to a complete block.
175   * 
176   * @return BlockInfo - a complete block.
177   * @throws IOException if the state of the block 
178   * (the generation stamp and the length) has not been committed by 
179   * the client or it does not have at least a minimal number of replicas 
180   * reported from data-nodes. 
181   */
182  BlockInfo convertToCompleteBlock() throws IOException {
183    assert getBlockUCState() != BlockUCState.COMPLETE :
184      "Trying to convert a COMPLETE block";
185    return new BlockInfo(this);
186  }
187
188  /** Set expected locations */
189  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
190    int numLocations = targets == null ? 0 : targets.length;
191    this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
192    for(int i = 0; i < numLocations; i++)
193      replicas.add(
194        new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
195  }
196
197  /**
198   * Create array of expected replica locations
199   * (as has been assigned by chooseTargets()).
200   */
201  public DatanodeStorageInfo[] getExpectedStorageLocations() {
202    int numLocations = replicas == null ? 0 : replicas.size();
203    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
204    for(int i = 0; i < numLocations; i++)
205      storages[i] = replicas.get(i).getExpectedStorageLocation();
206    return storages;
207  }
208
209  /** Get the number of expected locations */
210  public int getNumExpectedLocations() {
211    return replicas == null ? 0 : replicas.size();
212  }
213
214  /**
215   * Return the state of the block under construction.
216   * @see BlockUCState
217   */
218  @Override // BlockInfo
219  public BlockUCState getBlockUCState() {
220    return blockUCState;
221  }
222
223  void setBlockUCState(BlockUCState s) {
224    blockUCState = s;
225  }
226
227  /** Get block recovery ID */
228  public long getBlockRecoveryId() {
229    return blockRecoveryId;
230  }
231
232  /**
233   * Process the recorded replicas. When about to commit or finish the
234   * pipeline recovery sort out bad replicas.
235   * @param genStamp  The final generation stamp for the block.
236   */
237  public void setGenerationStampAndVerifyReplicas(long genStamp) {
238    // Set the generation stamp for the block.
239    setGenerationStamp(genStamp);
240    if (replicas == null)
241      return;
242
243    // Remove the replicas with wrong gen stamp.
244    // The replica list is unchanged.
245    for (ReplicaUnderConstruction r : replicas) {
246      if (genStamp != r.getGenerationStamp()) {
247        r.getExpectedStorageLocation().removeBlock(this);
248        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
249            + "from location: " + r.getExpectedStorageLocation());
250      }
251    }
252  }
253
254  /**
255   * Commit block's length and generation stamp as reported by the client.
256   * Set block state to {@link BlockUCState#COMMITTED}.
257   * @param block - contains client reported block length and generation 
258   * @throws IOException if block ids are inconsistent.
259   */
260  void commitBlock(Block block) throws IOException {
261    if(getBlockId() != block.getBlockId())
262      throw new IOException("Trying to commit inconsistent block: id = "
263          + block.getBlockId() + ", expected id = " + getBlockId());
264    blockUCState = BlockUCState.COMMITTED;
265    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
266    // Sort out invalid replicas.
267    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
268  }
269
270  /**
271   * Initialize lease recovery for this block.
272   * Find the first alive data-node starting from the previous primary and
273   * make it primary.
274   */
275  public void initializeBlockRecovery(long recoveryId) {
276    setBlockUCState(BlockUCState.UNDER_RECOVERY);
277    blockRecoveryId = recoveryId;
278    if (replicas.size() == 0) {
279      NameNode.blockStateChangeLog.warn("BLOCK*"
280        + " BlockInfoUnderConstruction.initLeaseRecovery:"
281        + " No blocks found, lease removed.");
282    }
283    boolean allLiveReplicasTriedAsPrimary = true;
284    for (int i = 0; i < replicas.size(); i++) {
285      // Check if all replicas have been tried or not.
286      if (replicas.get(i).isAlive()) {
287        allLiveReplicasTriedAsPrimary =
288            (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
289      }
290    }
291    if (allLiveReplicasTriedAsPrimary) {
292      // Just set all the replicas to be chosen whether they are alive or not.
293      for (int i = 0; i < replicas.size(); i++) {
294        replicas.get(i).setChosenAsPrimary(false);
295      }
296    }
297    long mostRecentLastUpdate = 0;
298    ReplicaUnderConstruction primary = null;
299    primaryNodeIndex = -1;
300    for(int i = 0; i < replicas.size(); i++) {
301      // Skip alive replicas which have been chosen for recovery.
302      if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
303        continue;
304      }
305      final ReplicaUnderConstruction ruc = replicas.get(i);
306      final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate(); 
307      if (lastUpdate > mostRecentLastUpdate) {
308        primaryNodeIndex = i;
309        primary = ruc;
310        mostRecentLastUpdate = lastUpdate;
311      }
312    }
313    if (primary != null) {
314      primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
315      primary.setChosenAsPrimary(true);
316      NameNode.blockStateChangeLog.info("BLOCK* " + this
317        + " recovery started, primary=" + primary);
318    }
319  }
320
321  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
322                     Block block,
323                     ReplicaState rState) {
324    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
325    while (it.hasNext()) {
326      ReplicaUnderConstruction r = it.next();
327      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
328      if(expectedLocation == storage) {
329        // Record the gen stamp from the report
330        r.setGenerationStamp(block.getGenerationStamp());
331        return;
332      } else if (expectedLocation != null &&
333                 expectedLocation.getDatanodeDescriptor() ==
334                     storage.getDatanodeDescriptor()) {
335
336        // The Datanode reported that the block is on a different storage
337        // than the one chosen by BlockPlacementPolicy. This can occur as
338        // we allow Datanodes to choose the target storage. Update our
339        // state by removing the stale entry and adding a new one.
340        it.remove();
341        break;
342      }
343    }
344    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
345  }
346
347  @Override // BlockInfo
348  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
349  public int hashCode() {
350    return super.hashCode();
351  }
352
353  @Override // BlockInfo
354  public boolean equals(Object obj) {
355    // Sufficient to rely on super's implementation
356    return (this == obj) || super.equals(obj);
357  }
358
359  @Override
360  public String toString() {
361    final StringBuilder b = new StringBuilder(100);
362    appendStringTo(b);
363    return b.toString();
364  }
365
366  @Override
367  public void appendStringTo(StringBuilder sb) {
368    super.appendStringTo(sb);
369    appendUCParts(sb);
370  }
371
372  private void appendUCParts(StringBuilder sb) {
373    sb.append("{blockUCState=").append(blockUCState)
374      .append(", primaryNodeIndex=").append(primaryNodeIndex)
375      .append(", replicas=[");
376    Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
377    if (iter.hasNext()) {
378      iter.next().appendStringTo(sb);
379      while (iter.hasNext()) {
380        sb.append(", ");
381        iter.next().appendStringTo(sb);
382      }
383    }
384    sb.append("]}");
385  }
386}