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.datanode;
019
020import java.io.File;
021
022import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
023import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
024import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
025
026/**
027 * This class represents replicas that are under block recovery
028 * It has a recovery id that is equal to the generation stamp 
029 * that the replica will be bumped to after recovery
030 * The recovery id is used to handle multiple concurrent block recoveries.
031 * A recovery with higher recovery id preempts recoveries with a lower id.
032 *
033 */
034public class ReplicaUnderRecovery extends ReplicaInfo {
035  private ReplicaInfo original; // the original replica that needs to be recovered
036  private long recoveryId; // recovery id; it is also the generation stamp 
037                           // that the replica will be bumped to after recovery
038
039  public ReplicaUnderRecovery(ReplicaInfo replica, long recoveryId) {
040    super(replica.getBlockId(), replica.getNumBytes(), replica.getGenerationStamp(),
041        replica.getVolume(), replica.getDir());
042    if ( replica.getState() != ReplicaState.FINALIZED &&
043         replica.getState() != ReplicaState.RBW &&
044         replica.getState() != ReplicaState.RWR ) {
045      throw new IllegalArgumentException("Cannot recover replica: " + replica);
046    }
047    this.original = replica;
048    this.recoveryId = recoveryId;
049  }
050
051  /**
052   * Copy constructor.
053   * @param from where to copy from
054   */
055  public ReplicaUnderRecovery(ReplicaUnderRecovery from) {
056    super(from);
057    this.original = from.getOriginalReplica();
058    this.recoveryId = from.getRecoveryID();
059  }
060
061  /** 
062   * Get the recovery id
063   * @return the generation stamp that the replica will be bumped to 
064   */
065  public long getRecoveryID() {
066    return recoveryId;
067  }
068
069  /** 
070   * Set the recovery id
071   * @param recoveryId the new recoveryId
072   */
073  public void setRecoveryID(long recoveryId) {
074    if (recoveryId > this.recoveryId) {
075      this.recoveryId = recoveryId;
076    } else {
077      throw new IllegalArgumentException("The new rcovery id: " + recoveryId
078          + " must be greater than the current one: " + this.recoveryId);
079    }
080  }
081
082  /**
083   * Get the original replica that's under recovery
084   * @return the original replica under recovery
085   */
086  public ReplicaInfo getOriginalReplica() {
087    return original;
088  }
089
090  @Override //ReplicaInfo
091  public boolean isUnlinked() {
092    return original.isUnlinked();
093  }
094
095  @Override //ReplicaInfo
096  public void setUnlinked() {
097    original.setUnlinked();
098  }
099  
100  @Override //ReplicaInfo
101  public ReplicaState getState() {
102    return ReplicaState.RUR;
103  }
104  
105  @Override
106  public long getVisibleLength() {
107    return original.getVisibleLength();
108  }
109
110  @Override
111  public long getBytesOnDisk() {
112    return original.getBytesOnDisk();
113  }
114
115  @Override  //org.apache.hadoop.hdfs.protocol.Block
116  public void setBlockId(long blockId) {
117    super.setBlockId(blockId);
118    original.setBlockId(blockId);
119  }
120
121  @Override //org.apache.hadoop.hdfs.protocol.Block
122  public void setGenerationStamp(long gs) {
123    super.setGenerationStamp(gs);
124    original.setGenerationStamp(gs);
125  }
126  
127  @Override //org.apache.hadoop.hdfs.protocol.Block
128  public void setNumBytes(long numBytes) {
129    super.setNumBytes(numBytes);
130    original.setNumBytes(numBytes);
131  }
132  
133  @Override //ReplicaInfo
134  public void setDir(File dir) {
135    super.setDir(dir);
136    original.setDir(dir);
137  }
138  
139  @Override //ReplicaInfo
140  void setVolume(FsVolumeSpi vol) {
141    super.setVolume(vol);
142    original.setVolume(vol);
143  }
144  
145  @Override  // Object
146  public boolean equals(Object o) {
147    return super.equals(o);
148  }
149  
150  @Override  // Object
151  public int hashCode() {
152    return super.hashCode();
153  }
154
155  @Override
156  public String toString() {
157    return super.toString()
158        + "\n  recoveryId=" + recoveryId
159        + "\n  original=" + original;
160  }
161
162  public ReplicaRecoveryInfo createInfo() {
163    return new ReplicaRecoveryInfo(original.getBlockId(), 
164        original.getBytesOnDisk(), original.getGenerationStamp(),
165        original.getState()); 
166  }
167}