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.protocol.Block;
023import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
024import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
025
026/**
027 * This class represents a replica that is waiting to be recovered.
028 * After a datanode restart, any replica in "rbw" directory is loaded
029 * as a replica waiting to be recovered.
030 * A replica waiting to be recovered does not provision read nor
031 * participates in any pipeline recovery. It will become outdated if its
032 * client continues to write or be recovered as a result of
033 * lease recovery.
034 */
035public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
036
037  /**
038   * Constructor
039   * @param blockId block id
040   * @param len replica length
041   * @param genStamp replica generation stamp
042   * @param vol volume where replica is located
043   * @param dir directory path where block and meta files are located
044   */
045  public ReplicaWaitingToBeRecovered(long blockId, long len, long genStamp,
046      FsVolumeSpi vol, File dir) {
047    super(blockId, len, genStamp, vol, dir);
048  }
049  
050  /**
051   * Constructor
052   * @param block a block
053   * @param vol volume where replica is located
054   * @param dir directory path where block and meta files are located
055   */
056  public ReplicaWaitingToBeRecovered(Block block, FsVolumeSpi vol, File dir) {
057    super(block, vol, dir);
058  }
059  
060  /**
061   * Copy constructor.
062   * @param from where to copy from
063   */
064  public ReplicaWaitingToBeRecovered(ReplicaWaitingToBeRecovered from) {
065    super(from);
066  }
067
068  @Override //ReplicaInfo
069  public ReplicaState getState() {
070    return ReplicaState.RWR;
071  }
072  
073  @Override //ReplicaInfo
074  public long getVisibleLength() {
075    return -1;  //no bytes are visible
076  }
077  
078  @Override
079  public long getBytesOnDisk() {
080    return getNumBytes();
081  }
082
083  @Override  // Object
084  public boolean equals(Object o) {
085    return super.equals(o);
086  }
087  
088  @Override  // Object
089  public int hashCode() {
090    return super.hashCode();
091  }
092
093  @Override
094  public String toString() {
095    return super.toString();
096  }
097}