You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wh...@apache.org on 2015/09/26 20:17:10 UTC

[01/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 b46e4ceaf -> 94cbb6d16


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
deleted file mode 100644
index 745ca7e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.EnumSet;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.crypto.CryptoOutputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.DFSOutputStream;
-
-import com.google.common.base.Preconditions;
-
-/**
- * The Hdfs implementation of {@link FSDataOutputStream}.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class HdfsDataOutputStream extends FSDataOutputStream {
-  public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats,
-      long startPosition) throws IOException {
-    super(out, stats, startPosition);
-  }
-
-  public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
-      ) throws IOException {
-    this(out, stats, 0L);
-  }
-
-  public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
-      long startPosition) throws IOException {
-    super(out, stats, startPosition);
-    Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
-        "CryptoOutputStream should wrap a DFSOutputStream");
-  }
-
-  public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
-      throws IOException {
-    this(out, stats, 0L);
-  }
-
-  /**
-   * Get the actual number of replicas of the current block.
-   * 
-   * This can be different from the designated replication factor of the file
-   * because the namenode does not maintain replication for the blocks which are
-   * currently being written to. Depending on the configuration, the client may
-   * continue to write to a block even if a few datanodes in the write pipeline
-   * have failed, or the client may add a new datanodes once a datanode has
-   * failed.
-   * 
-   * @return the number of valid replicas of the current block
-   */
-  public synchronized int getCurrentBlockReplication() throws IOException {
-    OutputStream wrappedStream = getWrappedStream();
-    if (wrappedStream instanceof CryptoOutputStream) {
-      wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
-    }
-    return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
-  }
-  
-  /**
-   * Sync buffered data to DataNodes (flush to disk devices).
-   * 
-   * @param syncFlags
-   *          Indicate the detailed semantic and actions of the hsync.
-   * @throws IOException
-   * @see FSDataOutputStream#hsync()
-   */
-  public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
-    OutputStream wrappedStream = getWrappedStream();
-    if (wrappedStream instanceof CryptoOutputStream) {
-      ((CryptoOutputStream) wrappedStream).flush();
-      wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
-    }
-    ((DFSOutputStream) wrappedStream).hsync(syncFlags);
-  }
-  
-  public static enum SyncFlag {
-
-    /**
-     * When doing sync to DataNodes, also update the metadata (block length) in
-     * the NameNode.
-     */
-    UPDATE_LENGTH,
-
-    /**
-     * Sync the data to DataNode, close the current block, and allocate a new
-     * block
-     */
-    END_BLOCK;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
deleted file mode 100644
index b41e2c3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
+++ /dev/null
@@ -1,524 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client.impl;
-
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSOutputStream;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Time;
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * <p>
- * Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing file-being-written leases
- * on the namenode.
- * When a file is opened for write (create or append),
- * namenode stores a file lease for recording the identity of the writer.
- * The writer (i.e. the DFSClient) is required to renew the lease periodically.
- * When the lease is not renewed before it expires,
- * the namenode considers the writer as failed and then it may either let
- * another writer to obtain the lease or close the file.
- * </p>
- * <p>
- * This class also provides the following functionality:
- * <ul>
- * <li>
- * It maintains a map from (namenode, user) pairs to lease renewers.
- * The same {@link LeaseRenewer} instance is used for renewing lease
- * for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and the same user.
- * </li>
- * <li>
- * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
- * Periodically the leases for all the clients are renewed.
- * A client is removed from the list when the client is closed.
- * </li>
- * <li>
- * A thread per namenode per user is used by the {@link LeaseRenewer}
- * to renew the leases.
- * </li>
- * </ul>
- * </p>
- */
-@InterfaceAudience.Private
-public class LeaseRenewer {
-  static final Log LOG = LogFactory.getLog(LeaseRenewer.class);
-
-  static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L;
-  static final long LEASE_RENEWER_SLEEP_DEFAULT = 1000L;
-
-  /** Get a {@link LeaseRenewer} instance */
-  public static LeaseRenewer getInstance(final String authority,
-      final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
-    final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
-    r.addClient(dfsc);
-    return r;
-  }
-
-  /**
-   * A factory for sharing {@link LeaseRenewer} objects
-   * among {@link DFSClient} instances
-   * so that there is only one renewer per authority per user.
-   */
-  private static class Factory {
-    private static final Factory INSTANCE = new Factory();
-
-    private static class Key {
-      /** Namenode info */
-      final String authority;
-      /** User info */
-      final UserGroupInformation ugi;
-
-      private Key(final String authority, final UserGroupInformation ugi) {
-        if (authority == null) {
-          throw new HadoopIllegalArgumentException("authority == null");
-        } else if (ugi == null) {
-          throw new HadoopIllegalArgumentException("ugi == null");
-        }
-
-        this.authority = authority;
-        this.ugi = ugi;
-      }
-
-      @Override
-      public int hashCode() {
-        return authority.hashCode() ^ ugi.hashCode();
-      }
-
-      @Override
-      public boolean equals(Object obj) {
-        if (obj == this) {
-          return true;
-        }
-        if (obj != null && obj instanceof Key) {
-          final Key that = (Key)obj;
-          return this.authority.equals(that.authority)
-                 && this.ugi.equals(that.ugi);
-        }
-        return false;
-      }
-
-      @Override
-      public String toString() {
-        return ugi.getShortUserName() + "@" + authority;
-      }
-    }
-
-    /** A map for per user per namenode renewers. */
-    private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
-
-    /** Get a renewer. */
-    private synchronized LeaseRenewer get(final String authority,
-        final UserGroupInformation ugi) {
-      final Key k = new Key(authority, ugi);
-      LeaseRenewer r = renewers.get(k);
-      if (r == null) {
-        r = new LeaseRenewer(k);
-        renewers.put(k, r);
-      }
-      return r;
-    }
-
-    /** Remove the given renewer. */
-    private synchronized void remove(final LeaseRenewer r) {
-      final LeaseRenewer stored = renewers.get(r.factorykey);
-      //Since a renewer may expire, the stored renewer can be different.
-      if (r == stored) {
-        if (!r.clientsRunning()) {
-          renewers.remove(r.factorykey);
-        }
-      }
-    }
-  }
-
-  /** The time in milliseconds that the map became empty. */
-  private long emptyTime = Long.MAX_VALUE;
-  /** A fixed lease renewal time period in milliseconds */
-  private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD / 2;
-
-  /** A daemon for renewing lease */
-  private Daemon daemon = null;
-  /** Only the daemon with currentId should run. */
-  private int currentId = 0;
-
-  /**
-   * A period in milliseconds that the lease renewer thread should run
-   * after the map became empty.
-   * In other words,
-   * if the map is empty for a time period longer than the grace period,
-   * the renewer should terminate.
-   */
-  private long gracePeriod;
-  /**
-   * The time period in milliseconds
-   * that the renewer sleeps for each iteration.
-   */
-  private long sleepPeriod;
-
-  private final Factory.Key factorykey;
-
-  /** A list of clients corresponding to this renewer. */
-  private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
-
-  /**
-   * A stringified stack trace of the call stack when the Lease Renewer
-   * was instantiated. This is only generated if trace-level logging is
-   * enabled on this class.
-   */
-  private final String instantiationTrace;
-
-  private LeaseRenewer(Factory.Key factorykey) {
-    this.factorykey = factorykey;
-    unsyncSetGraceSleepPeriod(LEASE_RENEWER_GRACE_DEFAULT);
-
-    if (LOG.isTraceEnabled()) {
-      instantiationTrace = StringUtils.stringifyException(
-        new Throwable("TRACE"));
-    } else {
-      instantiationTrace = null;
-    }
-  }
-
-  /** @return the renewal time in milliseconds. */
-  private synchronized long getRenewalTime() {
-    return renewal;
-  }
-
-  /** Used for testing only. */
-  @VisibleForTesting
-  public synchronized void setRenewalTime(final long renewal) {
-    this.renewal = renewal;
-  }
-
-  /** Add a client. */
-  private synchronized void addClient(final DFSClient dfsc) {
-    for(DFSClient c : dfsclients) {
-      if (c == dfsc) {
-        //client already exists, nothing to do.
-        return;
-      }
-    }
-    //client not found, add it
-    dfsclients.add(dfsc);
-
-    //update renewal time
-    final int hdfsTimeout = dfsc.getConf().getHdfsTimeout();
-    if (hdfsTimeout > 0) {
-      final long half = hdfsTimeout/2;
-      if (half < renewal) {
-        this.renewal = half;
-      }
-    }
-  }
-
-  private synchronized boolean clientsRunning() {
-    for(Iterator<DFSClient> i = dfsclients.iterator(); i.hasNext(); ) {
-      if (!i.next().isClientRunning()) {
-        i.remove();
-      }
-    }
-    return !dfsclients.isEmpty();
-  }
-
-  private synchronized long getSleepPeriod() {
-    return sleepPeriod;
-  }
-
-  /** Set the grace period and adjust the sleep period accordingly. */
-  synchronized void setGraceSleepPeriod(final long gracePeriod) {
-    unsyncSetGraceSleepPeriod(gracePeriod);
-  }
-
-  private void unsyncSetGraceSleepPeriod(final long gracePeriod) {
-    if (gracePeriod < 100L) {
-      throw new HadoopIllegalArgumentException(gracePeriod
-          + " = gracePeriod < 100ms is too small.");
-    }
-    this.gracePeriod = gracePeriod;
-    final long half = gracePeriod/2;
-    this.sleepPeriod = half < LEASE_RENEWER_SLEEP_DEFAULT?
-        half: LEASE_RENEWER_SLEEP_DEFAULT;
-  }
-
-  /** Is the daemon running? */
-  synchronized boolean isRunning() {
-    return daemon != null && daemon.isAlive();
-  }
-
-  /** Does this renewer have nothing to renew? */
-  public boolean isEmpty() {
-    return dfsclients.isEmpty();
-  }
-
-  /** Used only by tests */
-  synchronized String getDaemonName() {
-    return daemon.getName();
-  }
-
-  /** Is the empty period longer than the grace period? */
-  private synchronized boolean isRenewerExpired() {
-    return emptyTime != Long.MAX_VALUE
-        && Time.monotonicNow() - emptyTime > gracePeriod;
-  }
-
-  public synchronized void put(final long inodeId, final DFSOutputStream out,
-      final DFSClient dfsc) {
-    if (dfsc.isClientRunning()) {
-      if (!isRunning() || isRenewerExpired()) {
-        //start a new deamon with a new id.
-        final int id = ++currentId;
-        daemon = new Daemon(new Runnable() {
-          @Override
-          public void run() {
-            try {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Lease renewer daemon for " + clientsString()
-                    + " with renew id " + id + " started");
-              }
-              LeaseRenewer.this.run(id);
-            } catch(InterruptedException e) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
-                    + " is interrupted.", e);
-              }
-            } finally {
-              synchronized(LeaseRenewer.this) {
-                Factory.INSTANCE.remove(LeaseRenewer.this);
-              }
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Lease renewer daemon for " + clientsString()
-                    + " with renew id " + id + " exited");
-              }
-            }
-          }
-
-          @Override
-          public String toString() {
-            return String.valueOf(LeaseRenewer.this);
-          }
-        });
-        daemon.start();
-      }
-      dfsc.putFileBeingWritten(inodeId, out);
-      emptyTime = Long.MAX_VALUE;
-    }
-  }
-
-  @VisibleForTesting
-  synchronized void setEmptyTime(long time) {
-    emptyTime = time;
-  }
-
-  /** Close a file. */
-  public void closeFile(final long inodeId, final DFSClient dfsc) {
-    dfsc.removeFileBeingWritten(inodeId);
-
-    synchronized(this) {
-      if (dfsc.isFilesBeingWrittenEmpty()) {
-        dfsclients.remove(dfsc);
-      }
-      //update emptyTime if necessary
-      if (emptyTime == Long.MAX_VALUE) {
-        for(DFSClient c : dfsclients) {
-          if (!c.isFilesBeingWrittenEmpty()) {
-            //found a non-empty file-being-written map
-            return;
-          }
-        }
-        //discover the first time that all file-being-written maps are empty.
-        emptyTime = Time.monotonicNow();
-      }
-    }
-  }
-
-  /** Close the given client. */
-  public synchronized void closeClient(final DFSClient dfsc) {
-    dfsclients.remove(dfsc);
-    if (dfsclients.isEmpty()) {
-      if (!isRunning() || isRenewerExpired()) {
-        Factory.INSTANCE.remove(LeaseRenewer.this);
-        return;
-      }
-      if (emptyTime == Long.MAX_VALUE) {
-        //discover the first time that the client list is empty.
-        emptyTime = Time.monotonicNow();
-      }
-    }
-
-    //update renewal time
-    if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
-      long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
-      for(DFSClient c : dfsclients) {
-        final int timeout = c.getConf().getHdfsTimeout();
-        if (timeout > 0 && timeout < min) {
-          min = timeout;
-        }
-      }
-      renewal = min/2;
-    }
-  }
-
-  public void interruptAndJoin() throws InterruptedException {
-    Daemon daemonCopy = null;
-    synchronized (this) {
-      if (isRunning()) {
-        daemon.interrupt();
-        daemonCopy = daemon;
-      }
-    }
-
-    if (daemonCopy != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Wait for lease checker to terminate");
-      }
-      daemonCopy.join();
-    }
-  }
-
-  private void renew() throws IOException {
-    final List<DFSClient> copies;
-    synchronized(this) {
-      copies = new ArrayList<DFSClient>(dfsclients);
-    }
-    //sort the client names for finding out repeated names.
-    Collections.sort(copies, new Comparator<DFSClient>() {
-      @Override
-      public int compare(final DFSClient left, final DFSClient right) {
-        return left.getClientName().compareTo(right.getClientName());
-      }
-    });
-    String previousName = "";
-    for(int i = 0; i < copies.size(); i++) {
-      final DFSClient c = copies.get(i);
-      //skip if current client name is the same as the previous name.
-      if (!c.getClientName().equals(previousName)) {
-        if (!c.renewLease()) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Did not renew lease for client " +
-                c);
-          }
-          continue;
-        }
-        previousName = c.getClientName();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Lease renewed for client " + previousName);
-        }
-      }
-    }
-  }
-
-  /**
-   * Periodically check in with the namenode and renew all the leases
-   * when the lease period is half over.
-   */
-  private void run(final int id) throws InterruptedException {
-    for(long lastRenewed = Time.monotonicNow(); !Thread.interrupted();
-        Thread.sleep(getSleepPeriod())) {
-      final long elapsed = Time.monotonicNow() - lastRenewed;
-      if (elapsed >= getRenewalTime()) {
-        try {
-          renew();
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Lease renewer daemon for " + clientsString()
-                + " with renew id " + id + " executed");
-          }
-          lastRenewed = Time.monotonicNow();
-        } catch (SocketTimeoutException ie) {
-          LOG.warn("Failed to renew lease for " + clientsString() + " for "
-              + (elapsed/1000) + " seconds.  Aborting ...", ie);
-          synchronized (this) {
-            while (!dfsclients.isEmpty()) {
-              DFSClient dfsClient = dfsclients.get(0);
-              dfsClient.closeAllFilesBeingWritten(true);
-              closeClient(dfsClient);
-            }
-            //Expire the current LeaseRenewer thread.
-            emptyTime = 0;
-          }
-          break;
-        } catch (IOException ie) {
-          LOG.warn("Failed to renew lease for " + clientsString() + " for "
-              + (elapsed/1000) + " seconds.  Will retry shortly ...", ie);
-        }
-      }
-
-      synchronized(this) {
-        if (id != currentId || isRenewerExpired()) {
-          if (LOG.isDebugEnabled()) {
-            if (id != currentId) {
-              LOG.debug("Lease renewer daemon for " + clientsString()
-                  + " with renew id " + id + " is not current");
-            } else {
-               LOG.debug("Lease renewer daemon for " + clientsString()
-                  + " with renew id " + id + " expired");
-            }
-          }
-          //no longer the current daemon or expired
-          return;
-        }
-
-        // if no clients are in running state or there is no more clients
-        // registered with this renewer, stop the daemon after the grace
-        // period.
-        if (!clientsRunning() && emptyTime == Long.MAX_VALUE) {
-          emptyTime = Time.monotonicNow();
-        }
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    String s = getClass().getSimpleName() + ":" + factorykey;
-    if (LOG.isTraceEnabled()) {
-      return s + ", clients=" +  clientsString()
-        + ", created at " + instantiationTrace;
-    }
-    return s;
-  }
-
-  /** Get the names of all clients */
-  private synchronized String clientsString() {
-    if (dfsclients.isEmpty()) {
-      return "[]";
-    } else {
-      final StringBuilder b = new StringBuilder("[").append(
-          dfsclients.get(0).getClientName());
-      for(int i = 1; i < dfsclients.size(); i++) {
-        b.append(", ").append(dfsclients.get(i).getClientName());
-      }
-      return b.append("]").toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
deleted file mode 100644
index e4b51c5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.inotify;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class MissingEventsException extends Exception {
-  private static final long serialVersionUID = 1L;
-
-  private long expectedTxid;
-  private long actualTxid;
-
-  public MissingEventsException() {}
-
-  public MissingEventsException(long expectedTxid, long actualTxid) {
-    this.expectedTxid = expectedTxid;
-    this.actualTxid = actualTxid;
-  }
-
-  public long getExpectedTxid() {
-    return expectedTxid;
-  }
-
-  public long getActualTxid() {
-    return actualTxid;
-  }
-
-  @Override
-  public String toString() {
-    return "We expected the next batch of events to start with transaction ID "
-        + expectedTxid + ", but it instead started with transaction ID " +
-        actualTxid + ". Most likely the intervening transactions were cleaned "
-        + "up as part of checkpointing.";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
deleted file mode 100644
index 1210999..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Indicates a failure manipulating an ACL.
- */
-@InterfaceAudience.Private
-public class AclException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * Creates a new AclException.
-   *
-   * @param message String message
-   */
-  public AclException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
deleted file mode 100644
index 923cdb4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.hadoop.fs.InvalidRequestException;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.base.Preconditions;
-
-/**
- * CacheDirectiveIterator is a remote iterator that iterates cache directives.
- * It supports retrying in case of namenode failover.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class CacheDirectiveIterator
-    extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
-
-  private CacheDirectiveInfo filter;
-  private final ClientProtocol namenode;
-  private final Sampler<?> traceSampler;
-
-  public CacheDirectiveIterator(ClientProtocol namenode,
-      CacheDirectiveInfo filter, Sampler<?> traceSampler) {
-    super(0L);
-    this.namenode = namenode;
-    this.filter = filter;
-    this.traceSampler = traceSampler;
-  }
-
-  private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
-    CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder(filter);
-    builder.setId(null);
-    return builder.build();
-  }
-
-  /**
-   * Used for compatibility when communicating with a server version that
-   * does not support filtering directives by ID.
-   */
-  private static class SingleEntry implements
-      BatchedEntries<CacheDirectiveEntry> {
-
-    private final CacheDirectiveEntry entry;
-
-    public SingleEntry(final CacheDirectiveEntry entry) {
-      this.entry = entry;
-    }
-
-    @Override
-    public CacheDirectiveEntry get(int i) {
-      if (i > 0) {
-        return null;
-      }
-      return entry;
-    }
-
-    @Override
-    public int size() {
-      return 1;
-    }
-
-    @Override
-    public boolean hasMore() {
-      return false;
-    }
-  }
-
-  @Override
-  public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
-      throws IOException {
-    BatchedEntries<CacheDirectiveEntry> entries = null;
-    TraceScope scope = Trace.startSpan("listCacheDirectives", traceSampler);
-    try {
-      entries = namenode.listCacheDirectives(prevKey, filter);
-    } catch (IOException e) {
-      if (e.getMessage().contains("Filtering by ID is unsupported")) {
-        // Retry case for old servers, do the filtering client-side
-        long id = filter.getId();
-        filter = removeIdFromFilter(filter);
-        // Using id - 1 as prevId should get us a window containing the id
-        // This is somewhat brittle, since it depends on directives being
-        // returned in order of ascending ID.
-        entries = namenode.listCacheDirectives(id - 1, filter);
-        for (int i=0; i<entries.size(); i++) {
-          CacheDirectiveEntry entry = entries.get(i);
-          if (entry.getInfo().getId().equals((Long)id)) {
-            return new SingleEntry(entry);
-          }
-        }
-        throw new RemoteException(InvalidRequestException.class.getName(),
-            "Did not find requested id " + id);
-      }
-      throw e;
-    } finally {
-      scope.close();
-    }
-    Preconditions.checkNotNull(entries);
-    return entries;
-  }
-
-  @Override
-  public Long elementToPrevKey(CacheDirectiveEntry entry) {
-    return entry.getInfo().getId();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
deleted file mode 100644
index e9481f7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-/**
- * CachePoolIterator is a remote iterator that iterates cache pools.
- * It supports retrying in case of namenode failover.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class CachePoolIterator
-    extends BatchedRemoteIterator<String, CachePoolEntry> {
-
-  private final ClientProtocol namenode;
-  private final Sampler traceSampler;
-
-  public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) {
-    super("");
-    this.namenode = namenode;
-    this.traceSampler = traceSampler;
-  }
-
-  @Override
-  public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
-      throws IOException {
-    TraceScope scope = Trace.startSpan("listCachePools", traceSampler);
-    try {
-      return namenode.listCachePools(prevKey);
-    } finally {
-      scope.close();
-    }
-  }
-
-  @Override
-  public String elementToPrevKey(CachePoolEntry entry) {
-    return entry.getInfo().getPoolName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
deleted file mode 100644
index 0141215..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-/**
- * EncryptionZoneIterator is a remote iterator that iterates over encryption
- * zones. It supports retrying in case of namenode failover.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class EncryptionZoneIterator
-    extends BatchedRemoteIterator<Long, EncryptionZone> {
-
-  private final ClientProtocol namenode;
-  private final Sampler<?> traceSampler;
-
-  public EncryptionZoneIterator(ClientProtocol namenode,
-                                Sampler<?> traceSampler) {
-    super(Long.valueOf(0));
-    this.namenode = namenode;
-    this.traceSampler = traceSampler;
-  }
-
-  @Override
-  public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
-      throws IOException {
-    TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler);
-    try {
-      return namenode.listEncryptionZones(prevId);
-    } finally {
-      scope.close();
-    }
-  }
-
-  @Override
-  public Long elementToPrevKey(EncryptionZone entry) {
-    return entry.getId();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
deleted file mode 100644
index 25084c7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.StorageType;
-
-import static org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.long2String;
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class QuotaByStorageTypeExceededException extends QuotaExceededException {
-  protected static final long serialVersionUID = 1L;
-  protected StorageType type;
-
-  public QuotaByStorageTypeExceededException() {}
-
-  public QuotaByStorageTypeExceededException(String msg) {
-    super(msg);
-  }
-
-  public QuotaByStorageTypeExceededException(long quota, long count, StorageType type) {
-    super(quota, count);
-    this.type = type;
-  }
-
-  @Override
-  public String getMessage() {
-    String msg = super.getMessage();
-    if (msg == null) {
-      return "Quota by storage type : " + type.toString() +
-          " on path : " + (pathName==null ? "": pathName) +
-          " is exceeded. quota = "  + long2String(quota, "B", 2) +
-          " but space consumed = " + long2String(count, "B", 2);
-    } else {
-      return msg;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
deleted file mode 100644
index 03fb704..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Path;
-
-/** 
- * Thrown when a symbolic link is encountered in a path.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class UnresolvedPathException extends UnresolvedLinkException {
-  private static final long serialVersionUID = 1L;
-  private String path;        // The path containing the link
-  private String preceding;   // The path part preceding the link
-  private String remainder;   // The path part following the link
-  private String linkTarget;  // The link's target
-
-  /**
-   * Used by RemoteException to instantiate an UnresolvedPathException.
-   */
-  public UnresolvedPathException(String msg) {
-    super(msg);
-  }
-  
-  public UnresolvedPathException(String path, String preceding,
-      String remainder, String linkTarget) {
-    this.path = path;
-    this.preceding = preceding;
-    this.remainder = remainder;
-    this.linkTarget = linkTarget;
-  }
-
-  /**
-   * Return a path with the link resolved with the target.
-   */
-  public Path getResolvedPath() throws IOException {
-    // If the path is absolute we cam throw out the preceding part and
-    // just append the remainder to the target, otherwise append each
-    // piece to resolve the link in path.
-    boolean noRemainder = (remainder == null || "".equals(remainder));
-    Path target = new Path(linkTarget);
-    if (target.isUriPathAbsolute()) {
-      return noRemainder ? target : new Path(target, remainder);
-    } else {
-      return noRemainder
-        ? new Path(preceding, target)
-        : new Path(new Path(preceding, linkTarget), remainder);
-    }
-  }
-
-  @Override
-  public String getMessage() {
-    String msg = super.getMessage();
-    if (msg != null) {
-      return msg;
-    }
-    String myMsg = "Unresolved path " + path;
-    try {
-      return getResolvedPath().toString();
-    } catch (IOException e) {
-      // Ignore
-    }
-    return myMsg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
deleted file mode 100644
index c69986a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol.datatransfer;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-
-/**
- * The setting of replace-datanode-on-failure feature.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class ReplaceDatanodeOnFailure {
-  /** The replacement policies */
-  public enum Policy {
-    /** The feature is disabled in the entire site. */
-    DISABLE(Condition.FALSE),
-    /** Never add a new datanode. */
-    NEVER(Condition.FALSE),
-    /** @see ReplaceDatanodeOnFailure.Condition#DEFAULT */
-    DEFAULT(Condition.DEFAULT),
-    /** Always add a new datanode when an existing datanode is removed. */
-    ALWAYS(Condition.TRUE);
-
-    private final Condition condition;
-
-    private Policy(Condition condition) {
-      this.condition = condition;
-    }
-    
-    Condition getCondition() {
-      return condition;
-    }
-  }
-
-  /** Datanode replacement condition */
-  private static interface Condition {
-    /** Return true unconditionally. */
-    static final Condition TRUE = new Condition() {
-      @Override
-      public boolean satisfy(short replication, DatanodeInfo[] existings,
-          int nExistings, boolean isAppend, boolean isHflushed) {
-        return true;
-      }
-    };
-
-    /** Return false unconditionally. */
-    static final Condition FALSE = new Condition() {
-      @Override
-      public boolean satisfy(short replication, DatanodeInfo[] existings,
-          int nExistings, boolean isAppend, boolean isHflushed) {
-        return false;
-      }
-    };
-
-    /**
-     * DEFAULT condition:
-     *   Let r be the replication number.
-     *   Let n be the number of existing datanodes.
-     *   Add a new datanode only if r >= 3 and either
-     *   (1) floor(r/2) >= n; or
-     *   (2) r > n and the block is hflushed/appended.
-     */
-    static final Condition DEFAULT = new Condition() {
-      @Override
-      public boolean satisfy(final short replication,
-          final DatanodeInfo[] existings, final int n, final boolean isAppend,
-          final boolean isHflushed) {
-        if (replication < 3) {
-          return false;
-        } else {
-          if (n <= (replication/2)) {
-            return true;
-          } else {
-            return isAppend || isHflushed;
-          }
-        }
-      }
-    };
-
-    /** Is the condition satisfied? */
-    public boolean satisfy(short replication, DatanodeInfo[] existings,
-        int nExistings, boolean isAppend, boolean isHflushed);
-  }
-
-  private final Policy policy;
-  private final boolean bestEffort;
-  
-  public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
-    this.policy = policy;
-    this.bestEffort = bestEffort;
-  }
-
-  /** Check if the feature is enabled. */
-  public void checkEnabled() {
-    if (policy == Policy.DISABLE) {
-      throw new UnsupportedOperationException(
-          "This feature is disabled.  Please refer to "
-          + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY
-          + " configuration property.");
-    }
-  }
-
-  /**
-   * Best effort means that the client will try to replace the failed datanode
-   * (provided that the policy is satisfied), however, it will continue the
-   * write operation in case that the datanode replacement also fails.
-   * 
-   * @return Suppose the datanode replacement fails.
-   *     false: An exception should be thrown so that the write will fail.
-   *     true : The write should be resumed with the remaining datandoes.
-   */
-  public boolean isBestEffort() {
-    return bestEffort;
-  }
-
-  /** Does it need a replacement according to the policy? */
-  public boolean satisfy(
-      final short replication, final DatanodeInfo[] existings,
-      final boolean isAppend, final boolean isHflushed) {
-    final int n = existings == null? 0: existings.length;
-    if (n == 0 || n >= replication) {
-      //don't need to add datanode for any policy.
-      return false;
-    } else {
-      return policy.getCondition().satisfy(
-          replication, existings, n, isAppend, isHflushed);
-    }
-  }
-  
-  @Override
-  public String toString() {
-    return policy.toString();
-  }
-
-  /** Get the setting from configuration. */
-  public static ReplaceDatanodeOnFailure get(final Configuration conf) {
-    final Policy policy = getPolicy(conf);
-    final boolean bestEffort = conf.getBoolean(
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
-    
-    return new ReplaceDatanodeOnFailure(policy, bestEffort);
-  }
-
-  private static Policy getPolicy(final Configuration conf) {
-    final boolean enabled = conf.getBoolean(
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_DEFAULT);
-    if (!enabled) {
-      return Policy.DISABLE;
-    }
-
-    final String policy = conf.get(
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_DEFAULT);
-    for(int i = 1; i < Policy.values().length; i++) {
-      final Policy p = Policy.values()[i];
-      if (p.name().equalsIgnoreCase(policy)) {
-        return p;
-      }
-    }
-    throw new HadoopIllegalArgumentException("Illegal configuration value for "
-        + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
-        + ": " + policy);
-  }
-
-  /** Write the setting to configuration. */
-  public static void write(final Policy policy,
-      final boolean bestEffort, final Configuration conf) {
-    conf.setBoolean(
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
-        policy != Policy.DISABLE);
-    conf.set(
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
-        policy.name());
-    conf.setBoolean(
-        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
-        bestEffort);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index dc967ff..1afe0af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -879,7 +880,7 @@ public class Dispatcher {
     this.saslClient = new SaslDataTransferClient(conf,
         DataTransferSaslUtil.getSaslPropertiesResolver(conf),
         TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth);
-    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
+    this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
   }
 
   public DistributedFileSystem getDistributedFileSystem() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 957b2c7..eec2b2d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -38,7 +38,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
@@ -248,7 +248,7 @@ class BlockReceiver implements Closeable {
             out.getClass());
       }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
-          streams.getChecksumOut(), DFSUtil.getSmallBufferSize(
+          streams.getChecksumOut(), DFSUtilClient.getSmallBufferSize(
           datanode.getConf())));
       // write data chunk header if creating a new replica
       if (isCreate) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index fb8b132..d7e62bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -34,7 +34,7 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
@@ -111,7 +111,7 @@ class BlockSender implements java.io.Closeable {
   private static final int IO_FILE_BUFFER_SIZE;
   static {
     HdfsConfiguration conf = new HdfsConfiguration();
-    IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
+    IO_FILE_BUFFER_SIZE = DFSUtilClient.getIoFileBufferSize(conf);
   }
   private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
       IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index d20c444..e2d6490 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -109,6 +109,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.client.BlockReportOptions;
@@ -2186,7 +2187,7 @@ public class DataNode extends ReconfigurableBase
         unbufIn = saslStreams.in;
         
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            DFSUtil.getSmallBufferSize(conf)));
+            DFSUtilClient.getSmallBufferSize(conf)));
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
             false, false, true, DataNode.this, null, cachingStrategy);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 8c4e38a..665432e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -49,7 +49,7 @@ import java.util.Arrays;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -134,8 +134,8 @@ class DataXceiver extends Receiver implements Runnable {
     this.datanode = datanode;
     this.dataXceiverServer = dataXceiverServer;
     this.connectToDnViaHostname = datanode.getDnConf().connectToDnViaHostname;
-    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(datanode.getConf());
-    this.smallBufferSize = DFSUtil.getSmallBufferSize(datanode.getConf());
+    this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(datanode.getConf());
+    this.smallBufferSize = DFSUtilClient.getSmallBufferSize(datanode.getConf());
     remoteAddress = peer.getRemoteAddressString();
     final int colonIdx = remoteAddress.indexOf(':');
     remoteAddressWithoutPort =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
deleted file mode 100644
index b159d3a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.hadoop.hdfs.server.datanode;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-
-/**
- * Exception indicating that DataNode does not have a replica
- * that matches the target block.  
- */
-public class ReplicaNotFoundException extends IOException {
-  private static final long serialVersionUID = 1L;
-  public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
-  public final static String UNFINALIZED_REPLICA = 
-    "Cannot append to an unfinalized replica ";
-  public final static String UNFINALIZED_AND_NONRBW_REPLICA = 
-    "Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
-  public final static String NON_EXISTENT_REPLICA =
-    "Cannot append to a non-existent replica ";
-  public final static String UNEXPECTED_GS_REPLICA =
-    "Cannot append to a replica with unexpected generation stamp ";
-
-  public ReplicaNotFoundException() {
-    super();
-  }
-
-  public ReplicaNotFoundException(ExtendedBlock b) {
-    super("Replica not found for " + b);
-  }
-  
-  public ReplicaNotFoundException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 667a0b2..e12d69a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DU;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
@@ -111,7 +111,7 @@ class BlockPoolSlice {
       }
     }
 
-    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
+    this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
 
     this.deleteDuplicateReplicas = conf.getBoolean(
         DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 0d931c4..37b6b9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -59,7 +59,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -268,8 +268,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     this.datanode = datanode;
     this.dataStorage = storage;
     this.conf = conf;
-    this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
-    // The number of volumes required for operation is the total number 
+    this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
+    // The number of volumes required for operation is the total number
     // of volumes minus the number of failed volumes we can tolerate.
     volFailuresTolerated =
       conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
@@ -961,7 +961,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       File blockFile, int smallBufferSize, final Configuration conf)
       throws IOException {
     final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(srcMeta,
-        DFSUtil.getIoFileBufferSize(conf));
+        DFSUtilClient.getIoFileBufferSize(conf));
     final byte[] data = new byte[1 << 16];
     final byte[] crcs = new byte[checksum.getChecksumSize(data.length)];
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
index bf1fd98..476f596 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
@@ -240,7 +240,7 @@ class RamDiskAsyncLazyPersistService {
       boolean succeeded = false;
       final FsDatasetImpl dataset = (FsDatasetImpl)datanode.getFSDataset();
       try (FsVolumeReference ref = this.targetVolume) {
-        int smallBufferSize = DFSUtil.getSmallBufferSize(EMPTY_HDFS_CONF);
+        int smallBufferSize = DFSUtilClient.getSmallBufferSize(EMPTY_HDFS_CONF);
         // No FsDatasetImpl lock for the file copy
         File targetFiles[] = FsDatasetImpl.copyBlockFiles(
             blockId, genStamp, metaFile, blockFile, lazyPersistDir, true,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
deleted file mode 100644
index 0bdd2a5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public class RetryStartFileException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public RetryStartFileException() {
-    super("Preconditions for creating a file failed because of a " +
-        "transient error, retry create later.");
-  }
-
-  public RetryStartFileException(String s) {
-    super(s);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index 9783cca..f1b74f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -89,7 +89,7 @@ public class TransferFsImage {
     connectionFactory = URLConnectionFactory
         .newDefaultURLConnectionFactory(conf);
     isSpnegoEnabled = UserGroupInformation.isSecurityEnabled();
-    IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
+    IO_FILE_BUFFER_SIZE = DFSUtilClient.getIoFileBufferSize(conf);
   }
 
   private static final Log LOG = LogFactory.getLog(TransferFsImage.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
index 0b0da21..52cd9c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.IOException;
 import java.util.Random;
 
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fi.DataTransferTestUtil;
 import org.apache.hadoop.fi.DataTransferTestUtil.CountdownDoosAction;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index e324557..b236bff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -990,7 +990,7 @@ public class DFSTestUtil {
     final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
         NetUtils.getOutputStream(s, writeTimeout),
-        DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
+        DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
     // send the request

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
index b9682a1..5c011e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter;
-import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.net.NetUtils;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index c3f89ea..53b8038 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.*;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
index ca98441..b83157d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
@@ -124,8 +124,8 @@ public class TestClientProtocolForPipelineRecovery {
   public void testPipelineRecoveryForLastBlock() throws IOException {
     DFSClientFaultInjector faultInjector
         = Mockito.mock(DFSClientFaultInjector.class);
-    DFSClientFaultInjector oldInjector = DFSClientFaultInjector.instance;
-    DFSClientFaultInjector.instance = faultInjector;
+    DFSClientFaultInjector oldInjector = DFSClientFaultInjector.get();
+    DFSClientFaultInjector.set(faultInjector);
     Configuration conf = new HdfsConfiguration();
 
     conf.setInt(HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 3);
@@ -153,7 +153,7 @@ public class TestClientProtocolForPipelineRecovery {
             + " corrupt replicas.");
       }
     } finally {
-      DFSClientFaultInjector.instance = oldInjector;
+      DFSClientFaultInjector.set(oldInjector);
       if (cluster != null) {
         cluster.shutdown();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
index 2b11fbd..3850ff2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
@@ -76,7 +76,7 @@ public class TestCrcCorruption {
   @Before
   public void setUp() throws IOException {
     faultInjector = Mockito.mock(DFSClientFaultInjector.class);
-    DFSClientFaultInjector.instance = faultInjector;
+    DFSClientFaultInjector.set(faultInjector);
   }
 
   /** 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index a821c30..3435b7f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -903,16 +903,16 @@ public class TestDFSUtil {
     Configuration conf = new Configuration(false);
     conf.unset(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI);
     assertFalse("encryption enabled on no provider key",
-        DFSUtil.isHDFSEncryptionEnabled(conf));
+        DFSUtilClient.isHDFSEncryptionEnabled(conf));
     conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
     assertFalse("encryption enabled on empty provider key",
-        DFSUtil.isHDFSEncryptionEnabled(conf));
+        DFSUtilClient.isHDFSEncryptionEnabled(conf));
     conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "\n\t\n");
     assertFalse("encryption enabled on whitespace provider key",
-        DFSUtil.isHDFSEncryptionEnabled(conf));
+        DFSUtilClient.isHDFSEncryptionEnabled(conf));
     conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "http://hadoop.apache.org");
     assertTrue("encryption disabled on valid provider key",
-        DFSUtil.isHDFSEncryptionEnabled(conf));
+        DFSUtilClient.isHDFSEncryptionEnabled(conf));
 
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
index acb179c..e3dcd8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
@@ -90,7 +90,7 @@ public class TestFileStatus {
       int fileSize, int blockSize) throws IOException {
     // Create and write a file that contains three blocks of data
     FSDataOutputStream stm = fileSys.create(name, true,
-        DFSUtil.getIoFileBufferSize(conf), (short)repl, (long)blockSize);
+        DFSUtilClient.getIoFileBufferSize(conf), (short)repl, (long)blockSize);
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index ee3ef89..cc0fb92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -295,9 +295,8 @@ public class TestPread {
         hedgedReadTimeoutMillis);
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 0);
     // Set up the InjectionHandler
-    DFSClientFaultInjector.instance = Mockito
-        .mock(DFSClientFaultInjector.class);
-    DFSClientFaultInjector injector = DFSClientFaultInjector.instance;
+    DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
+    DFSClientFaultInjector injector = DFSClientFaultInjector.get();
     final int sleepMs = 100;
     Mockito.doAnswer(new Answer<Void>() {
       @Override
@@ -371,9 +370,8 @@ public class TestPread {
         initialHedgedReadTimeoutMillis);
 
     // Set up the InjectionHandler
-    DFSClientFaultInjector.instance = Mockito
-        .mock(DFSClientFaultInjector.class);
-    DFSClientFaultInjector injector = DFSClientFaultInjector.instance;
+    DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
+    DFSClientFaultInjector injector = DFSClientFaultInjector.get();
     // make preads sleep for 50ms
     Mockito.doAnswer(new Answer<Void>() {
       @Override


[06/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
deleted file mode 100644
index c9add53..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ /dev/null
@@ -1,892 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.net.InetSocketAddress;
-import java.util.List;
-
-import com.google.common.io.ByteArrayDataOutput;
-import com.google.common.io.ByteStreams;
-import org.apache.commons.lang.mutable.MutableBoolean;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
-import org.apache.hadoop.hdfs.net.DomainPeer;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplicaInfo;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.unix.DomainSocket;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.PerformanceAdvisory;
-import org.apache.hadoop.util.Time;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-
-/** 
- * Utility class to create BlockReader implementations.
- */
-@InterfaceAudience.Private
-public class BlockReaderFactory implements ShortCircuitReplicaCreator {
-  static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
-
-  public static class FailureInjector {
-    public void injectRequestFileDescriptorsFailure() throws IOException {
-      // do nothing
-    }
-    public boolean getSupportsReceiptVerification() {
-      return true;
-    }
-  }
-
-  @VisibleForTesting
-  static ShortCircuitReplicaCreator
-      createShortCircuitReplicaInfoCallback = null;
-
-  private final DfsClientConf conf;
-
-  /**
-   * Injects failures into specific operations during unit tests.
-   */
-  private static FailureInjector failureInjector = new FailureInjector();
-
-  /**
-   * The file name, for logging and debugging purposes.
-   */
-  private String fileName;
-
-  /**
-   * The block ID and block pool ID to use.
-   */
-  private ExtendedBlock block;
-
-  /**
-   * The block token to use for security purposes.
-   */
-  private Token<BlockTokenIdentifier> token;
-
-  /**
-   * The offset within the block to start reading at.
-   */
-  private long startOffset;
-
-  /**
-   * If false, we won't try to verify the block checksum.
-   */
-  private boolean verifyChecksum;
-
-  /**
-   * The name of this client.
-   */
-  private String clientName; 
-
-  /**
-   * The DataNode we're talking to.
-   */
-  private DatanodeInfo datanode;
-
-  /**
-   * StorageType of replica on DataNode.
-   */
-  private StorageType storageType;
-
-  /**
-   * If false, we won't try short-circuit local reads.
-   */
-  private boolean allowShortCircuitLocalReads;
-
-  /**
-   * The ClientContext to use for things like the PeerCache.
-   */
-  private ClientContext clientContext;
-
-  /**
-   * Number of bytes to read.  -1 indicates no limit.
-   */
-  private long length = -1;
-
-  /**
-   * Caching strategy to use when reading the block.
-   */
-  private CachingStrategy cachingStrategy;
-
-  /**
-   * Socket address to use to connect to peer.
-   */
-  private InetSocketAddress inetSocketAddress;
-
-  /**
-   * Remote peer factory to use to create a peer, if needed.
-   */
-  private RemotePeerFactory remotePeerFactory;
-
-  /**
-   * UserGroupInformation  to use for legacy block reader local objects, if needed.
-   */
-  private UserGroupInformation userGroupInformation;
-
-  /**
-   * Configuration to use for legacy block reader local objects, if needed.
-   */
-  private Configuration configuration;
-
-  /**
-   * Information about the domain socket path we should use to connect to the
-   * local peer-- or null if we haven't examined the local domain socket.
-   */
-  private DomainSocketFactory.PathInfo pathInfo;
-
-  /**
-   * The remaining number of times that we'll try to pull a socket out of the
-   * cache.
-   */
-  private int remainingCacheTries;
-
-  public BlockReaderFactory(DfsClientConf conf) {
-    this.conf = conf;
-    this.remainingCacheTries = conf.getNumCachedConnRetry();
-  }
-
-  public BlockReaderFactory setFileName(String fileName) {
-    this.fileName = fileName;
-    return this;
-  }
-
-  public BlockReaderFactory setBlock(ExtendedBlock block) {
-    this.block = block;
-    return this;
-  }
-
-  public BlockReaderFactory setBlockToken(Token<BlockTokenIdentifier> token) {
-    this.token = token;
-    return this;
-  }
-
-  public BlockReaderFactory setStartOffset(long startOffset) {
-    this.startOffset = startOffset;
-    return this;
-  }
-
-  public BlockReaderFactory setVerifyChecksum(boolean verifyChecksum) {
-    this.verifyChecksum = verifyChecksum;
-    return this;
-  }
-
-  public BlockReaderFactory setClientName(String clientName) {
-    this.clientName = clientName;
-    return this;
-  }
-
-  public BlockReaderFactory setDatanodeInfo(DatanodeInfo datanode) {
-    this.datanode = datanode;
-    return this;
-  }
-
-  public BlockReaderFactory setStorageType(StorageType storageType) {
-    this.storageType = storageType;
-    return this;
-  }
-
-  public BlockReaderFactory setAllowShortCircuitLocalReads(
-      boolean allowShortCircuitLocalReads) {
-    this.allowShortCircuitLocalReads = allowShortCircuitLocalReads;
-    return this;
-  }
-
-  public BlockReaderFactory setClientCacheContext(
-      ClientContext clientContext) {
-    this.clientContext = clientContext;
-    return this;
-  }
-
-  public BlockReaderFactory setLength(long length) {
-    this.length = length;
-    return this;
-  }
-
-  public BlockReaderFactory setCachingStrategy(
-      CachingStrategy cachingStrategy) {
-    this.cachingStrategy = cachingStrategy;
-    return this;
-  }
-
-  public BlockReaderFactory setInetSocketAddress (
-      InetSocketAddress inetSocketAddress) {
-    this.inetSocketAddress = inetSocketAddress;
-    return this;
-  }
-
-  public BlockReaderFactory setUserGroupInformation(
-      UserGroupInformation userGroupInformation) {
-    this.userGroupInformation = userGroupInformation;
-    return this;
-  }
-
-  public BlockReaderFactory setRemotePeerFactory(
-      RemotePeerFactory remotePeerFactory) {
-    this.remotePeerFactory = remotePeerFactory;
-    return this;
-  }
-
-  public BlockReaderFactory setConfiguration(
-      Configuration configuration) {
-    this.configuration = configuration;
-    return this;
-  }
-
-  @VisibleForTesting
-  public static void setFailureInjectorForTesting(FailureInjector injector) {
-    failureInjector = injector;
-  }
-
-  /**
-   * Build a BlockReader with the given options.
-   *
-   * This function will do the best it can to create a block reader that meets
-   * all of our requirements.  We prefer short-circuit block readers
-   * (BlockReaderLocal and BlockReaderLocalLegacy) over remote ones, since the
-   * former avoid the overhead of socket communication.  If short-circuit is
-   * unavailable, our next fallback is data transfer over UNIX domain sockets,
-   * if dfs.client.domain.socket.data.traffic has been enabled.  If that doesn't
-   * work, we will try to create a remote block reader that operates over TCP
-   * sockets.
-   *
-   * There are a few caches that are important here.
-   *
-   * The ShortCircuitCache stores file descriptor objects which have been passed
-   * from the DataNode. 
-   *
-   * The DomainSocketFactory stores information about UNIX domain socket paths
-   * that we not been able to use in the past, so that we don't waste time
-   * retrying them over and over.  (Like all the caches, it does have a timeout,
-   * though.)
-   *
-   * The PeerCache stores peers that we have used in the past.  If we can reuse
-   * one of these peers, we avoid the overhead of re-opening a socket.  However,
-   * if the socket has been timed out on the remote end, our attempt to reuse
-   * the socket may end with an IOException.  For that reason, we limit our
-   * attempts at socket reuse to dfs.client.cached.conn.retry times.  After
-   * that, we create new sockets.  This avoids the problem where a thread tries
-   * to talk to a peer that it hasn't talked to in a while, and has to clean out
-   * every entry in a socket cache full of stale entries.
-   *
-   * @return The new BlockReader.  We will not return null.
-   *
-   * @throws InvalidToken
-   *             If the block token was invalid.
-   *         InvalidEncryptionKeyException
-   *             If the encryption key was invalid.
-   *         Other IOException
-   *             If there was another problem.
-   */
-  public BlockReader build() throws IOException {
-    Preconditions.checkNotNull(configuration);
-    BlockReader reader = tryToCreateExternalBlockReader();
-    if (reader != null) {
-      return reader;
-    }
-    final ShortCircuitConf scConf = conf.getShortCircuitConf();
-    if (scConf.isShortCircuitLocalReads() && allowShortCircuitLocalReads) {
-      if (clientContext.getUseLegacyBlockReaderLocal()) {
-        reader = getLegacyBlockReaderLocal();
-        if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new legacy block reader local.");
-          }
-          return reader;
-        }
-      } else {
-        reader = getBlockReaderLocal();
-        if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new block reader local.");
-          }
-          return reader;
-        }
-      }
-    }
-    if (scConf.isDomainSocketDataTraffic()) {
-      reader = getRemoteBlockReaderFromDomain();
-      if (reader != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": returning new remote block reader using " +
-              "UNIX domain socket on " + pathInfo.getPath());
-        }
-        return reader;
-      }
-    }
-    Preconditions.checkState(!DFSInputStream.tcpReadsDisabledForTesting,
-        "TCP reads were disabled for testing, but we failed to " +
-        "do a non-TCP read.");
-    return getRemoteBlockReaderFromTcp();
-  }
-
-  private BlockReader tryToCreateExternalBlockReader() {
-    List<Class<? extends ReplicaAccessorBuilder>> clses =
-        conf.getReplicaAccessorBuilderClasses();
-    for (Class<? extends ReplicaAccessorBuilder> cls : clses) {
-      try {
-        ByteArrayDataOutput bado = ByteStreams.newDataOutput();
-        token.write(bado);
-        byte tokenBytes[] = bado.toByteArray();
-
-        Constructor<? extends ReplicaAccessorBuilder> ctor =
-            cls.getConstructor();
-        ReplicaAccessorBuilder builder = ctor.newInstance();
-        ReplicaAccessor accessor = builder.
-            setAllowShortCircuitReads(allowShortCircuitLocalReads).
-            setBlock(block.getBlockId(), block.getBlockPoolId()).
-            setGenerationStamp(block.getGenerationStamp()).
-            setBlockAccessToken(tokenBytes).
-            setClientName(clientName).
-            setConfiguration(configuration).
-            setFileName(fileName).
-            setVerifyChecksum(verifyChecksum).
-            setVisibleLength(length).
-            build();
-        if (accessor == null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": No ReplicaAccessor created by " +
-                cls.getName());
-          }
-        } else {
-          return new ExternalBlockReader(accessor, length, startOffset);
-        }
-      } catch (Throwable t) {
-        LOG.warn("Failed to construct new object of type " +
-            cls.getName(), t);
-      }
-    }
-    return null;
-  }
-
-
-  /**
-   * Get {@link BlockReaderLocalLegacy} for short circuited local reads.
-   * This block reader implements the path-based style of local reads
-   * first introduced in HDFS-2246.
-   */
-  private BlockReader getLegacyBlockReaderLocal() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
-    }
-    if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
-            "the address " + inetSocketAddress + " is not local");
-      }
-      return null;
-    }
-    if (clientContext.getDisableLegacyBlockReaderLocal()) {
-        PerformanceAdvisory.LOG.debug("{}: can't construct " +
-            "BlockReaderLocalLegacy because " +
-            "disableLegacyBlockReaderLocal is set.", this);
-      return null;
-    }
-    IOException ioe;
-    try {
-      return BlockReaderLocalLegacy.newBlockReader(conf,
-          userGroupInformation, configuration, fileName, block, token,
-          datanode, startOffset, length, storageType);
-    } catch (RemoteException remoteException) {
-      ioe = remoteException.unwrapRemoteException(
-                InvalidToken.class, AccessControlException.class);
-    } catch (IOException e) {
-      ioe = e;
-    }
-    if ((!(ioe instanceof AccessControlException)) &&
-        isSecurityException(ioe)) {
-      // Handle security exceptions.
-      // We do not handle AccessControlException here, since
-      // BlockReaderLocalLegacy#newBlockReader uses that exception to indicate
-      // that the user is not in dfs.block.local-path-access.user, a condition
-      // which requires us to disable legacy SCR.
-      throw ioe;
-    }
-    LOG.warn(this + ": error creating legacy BlockReaderLocal.  " +
-        "Disabling legacy local reads.", ioe);
-    clientContext.setDisableLegacyBlockReaderLocal();
-    return null;
-  }
-
-  private BlockReader getBlockReaderLocal() throws InvalidToken {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct a BlockReaderLocal " +
-          "for short-circuit reads.");
-    }
-    if (pathInfo == null) {
-      pathInfo = clientContext.getDomainSocketFactory()
-          .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
-    }
-    if (!pathInfo.getPathState().getUsableForShortCircuit()) {
-      PerformanceAdvisory.LOG.debug("{}: {} is not usable for short circuit; " +
-              "giving up on BlockReaderLocal.", this, pathInfo);
-      return null;
-    }
-    ShortCircuitCache cache = clientContext.getShortCircuitCache();
-    ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
-    ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
-    InvalidToken exc = info.getInvalidTokenException();
-    if (exc != null) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": got InvalidToken exception while trying to " +
-            "construct BlockReaderLocal via " + pathInfo.getPath());
-      }
-      throw exc;
-    }
-    if (info.getReplica() == null) {
-      PerformanceAdvisory.LOG.debug("{}: failed to get " +
-          "ShortCircuitReplica. Cannot construct " +
-          "BlockReaderLocal via {}", this, pathInfo.getPath());
-      return null;
-    }
-    return new BlockReaderLocal.Builder(conf.getShortCircuitConf()).
-        setFilename(fileName).
-        setBlock(block).
-        setStartOffset(startOffset).
-        setShortCircuitReplica(info.getReplica()).
-        setVerifyChecksum(verifyChecksum).
-        setCachingStrategy(cachingStrategy).
-        setStorageType(storageType).
-        build();
-  }
-
-  /**
-   * Fetch a pair of short-circuit block descriptors from a local DataNode.
-   *
-   * @return    Null if we could not communicate with the datanode,
-   *            a new ShortCircuitReplicaInfo object otherwise.
-   *            ShortCircuitReplicaInfo objects may contain either an InvalidToken
-   *            exception, or a ShortCircuitReplica object ready to use.
-   */
-  @Override
-  public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
-    if (createShortCircuitReplicaInfoCallback != null) {
-      ShortCircuitReplicaInfo info =
-        createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
-      if (info != null) return info;
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
-    }
-    BlockReaderPeer curPeer;
-    while (true) {
-      curPeer = nextDomainPeer();
-      if (curPeer == null) break;
-      if (curPeer.fromCache) remainingCacheTries--;
-      DomainPeer peer = (DomainPeer)curPeer.peer;
-      Slot slot = null;
-      ShortCircuitCache cache = clientContext.getShortCircuitCache();
-      try {
-        MutableBoolean usedPeer = new MutableBoolean(false);
-        slot = cache.allocShmSlot(datanode, peer, usedPeer,
-            new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
-            clientName);
-        if (usedPeer.booleanValue()) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": allocShmSlot used up our previous socket " +
-              peer.getDomainSocket() + ".  Allocating a new one...");
-          }
-          curPeer = nextDomainPeer();
-          if (curPeer == null) break;
-          peer = (DomainPeer)curPeer.peer;
-        }
-        ShortCircuitReplicaInfo info = requestFileDescriptors(peer, slot);
-        clientContext.getPeerCache().put(datanode, peer);
-        return info;
-      } catch (IOException e) {
-        if (slot != null) {
-          cache.freeSlot(slot);
-        }
-        if (curPeer.fromCache) {
-          // Handle an I/O error we got when using a cached socket.
-          // These are considered less serious, because the socket may be stale.
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(this + ": closing stale domain peer " + peer, e);
-          }
-          IOUtils.cleanup(LOG, peer);
-        } else {
-          // Handle an I/O error we got when using a newly created socket.
-          // We temporarily disable the domain socket path for a few minutes in
-          // this case, to prevent wasting more time on it.
-          LOG.warn(this + ": I/O error requesting file descriptors.  " + 
-              "Disabling domain socket " + peer.getDomainSocket(), e);
-          IOUtils.cleanup(LOG, peer);
-          clientContext.getDomainSocketFactory()
-              .disableDomainSocketPath(pathInfo.getPath());
-          return null;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Request file descriptors from a DomainPeer.
-   *
-   * @param peer   The peer to use for communication.
-   * @param slot   If non-null, the shared memory slot to associate with the 
-   *               new ShortCircuitReplica.
-   * 
-   * @return  A ShortCircuitReplica object if we could communicate with the
-   *          datanode; null, otherwise. 
-   * @throws  IOException If we encountered an I/O exception while communicating
-   *          with the datanode.
-   */
-  private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
-          Slot slot) throws IOException {
-    ShortCircuitCache cache = clientContext.getShortCircuitCache();
-    final DataOutputStream out =
-        new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
-    SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1,
-        failureInjector.getSupportsReceiptVerification());
-    DataInputStream in = new DataInputStream(peer.getInputStream());
-    BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
-        PBHelperClient.vintPrefixed(in));
-    DomainSocket sock = peer.getDomainSocket();
-    failureInjector.injectRequestFileDescriptorsFailure();
-    switch (resp.getStatus()) {
-    case SUCCESS:
-      byte buf[] = new byte[1];
-      FileInputStream fis[] = new FileInputStream[2];
-      sock.recvFileInputStreams(fis, buf, 0, buf.length);
-      ShortCircuitReplica replica = null;
-      try {
-        ExtendedBlockId key =
-            new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
-        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
-          LOG.trace("Sending receipt verification byte for slot " + slot);
-          sock.getOutputStream().write(0);
-        }
-        replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
-            Time.monotonicNow(), slot);
-        return new ShortCircuitReplicaInfo(replica);
-      } catch (IOException e) {
-        // This indicates an error reading from disk, or a format error.  Since
-        // it's not a socket communication problem, we return null rather than
-        // throwing an exception.
-        LOG.warn(this + ": error creating ShortCircuitReplica.", e);
-        return null;
-      } finally {
-        if (replica == null) {
-          IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
-        }
-      }
-    case ERROR_UNSUPPORTED:
-      if (!resp.hasShortCircuitAccessVersion()) {
-        LOG.warn("short-circuit read access is disabled for " +
-            "DataNode " + datanode + ".  reason: " + resp.getMessage());
-        clientContext.getDomainSocketFactory()
-            .disableShortCircuitForPath(pathInfo.getPath());
-      } else {
-        LOG.warn("short-circuit read access for the file " +
-            fileName + " is disabled for DataNode " + datanode +
-            ".  reason: " + resp.getMessage());
-      }
-      return null;
-    case ERROR_ACCESS_TOKEN:
-      String msg = "access control error while " +
-          "attempting to set up short-circuit access to " +
-          fileName + resp.getMessage();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(this + ":" + msg);
-      }
-      return new ShortCircuitReplicaInfo(new InvalidToken(msg));
-    default:
-      LOG.warn(this + ": unknown response code " + resp.getStatus() +
-          " while attempting to set up short-circuit access. " +
-          resp.getMessage());
-      clientContext.getDomainSocketFactory()
-          .disableShortCircuitForPath(pathInfo.getPath());
-      return null;
-    }
-  }
-
-  /**
-   * Get a RemoteBlockReader that communicates over a UNIX domain socket.
-   *
-   * @return The new BlockReader, or null if we failed to create the block
-   * reader.
-   *
-   * @throws InvalidToken    If the block token was invalid.
-   * Potentially other security-related execptions.
-   */
-  private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
-    if (pathInfo == null) {
-      pathInfo = clientContext.getDomainSocketFactory()
-          .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
-    }
-    if (!pathInfo.getPathState().getUsableForDataTransfer()) {
-      PerformanceAdvisory.LOG.debug("{}: not trying to create a " +
-          "remote block reader because the UNIX domain socket at {}" +
-           " is not usable.", this, pathInfo);
-      return null;
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create a remote block reader from the " +
-          "UNIX domain socket at " + pathInfo.getPath());
-    }
-
-    while (true) {
-      BlockReaderPeer curPeer = nextDomainPeer();
-      if (curPeer == null) break;
-      if (curPeer.fromCache) remainingCacheTries--;
-      DomainPeer peer = (DomainPeer)curPeer.peer;
-      BlockReader blockReader = null;
-      try {
-        blockReader = getRemoteBlockReader(peer);
-        return blockReader;
-      } catch (IOException ioe) {
-        IOUtils.cleanup(LOG, peer);
-        if (isSecurityException(ioe)) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": got security exception while constructing " +
-                "a remote block reader from the unix domain socket at " +
-                pathInfo.getPath(), ioe);
-          }
-          throw ioe;
-        }
-        if (curPeer.fromCache) {
-          // Handle an I/O error we got when using a cached peer.  These are
-          // considered less serious, because the underlying socket may be stale.
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Closed potentially stale domain peer " + peer, ioe);
-          }
-        } else {
-          // Handle an I/O error we got when using a newly created domain peer.
-          // We temporarily disable the domain socket path for a few minutes in
-          // this case, to prevent wasting more time on it.
-          LOG.warn("I/O error constructing remote block reader.  Disabling " +
-              "domain socket " + peer.getDomainSocket(), ioe);
-          clientContext.getDomainSocketFactory()
-              .disableDomainSocketPath(pathInfo.getPath());
-          return null;
-        }
-      } finally {
-        if (blockReader == null) {
-          IOUtils.cleanup(LOG, peer);
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Get a RemoteBlockReader that communicates over a TCP socket.
-   *
-   * @return The new BlockReader.  We will not return null, but instead throw
-   *         an exception if this fails.
-   *
-   * @throws InvalidToken
-   *             If the block token was invalid.
-   *         InvalidEncryptionKeyException
-   *             If the encryption key was invalid.
-   *         Other IOException
-   *             If there was another problem.
-   */
-  private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create a remote block reader from a " +
-          "TCP socket");
-    }
-    BlockReader blockReader = null;
-    while (true) {
-      BlockReaderPeer curPeer = null;
-      Peer peer = null;
-      try {
-        curPeer = nextTcpPeer();
-        if (curPeer.fromCache) remainingCacheTries--;
-        peer = curPeer.peer;
-        blockReader = getRemoteBlockReader(peer);
-        return blockReader;
-      } catch (IOException ioe) {
-        if (isSecurityException(ioe)) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": got security exception while constructing " +
-                "a remote block reader from " + peer, ioe);
-          }
-          throw ioe;
-        }
-        if ((curPeer != null) && curPeer.fromCache) {
-          // Handle an I/O error we got when using a cached peer.  These are
-          // considered less serious, because the underlying socket may be
-          // stale.
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Closed potentially stale remote peer " + peer, ioe);
-          }
-        } else {
-          // Handle an I/O error we got when using a newly created peer.
-          LOG.warn("I/O error constructing remote block reader.", ioe);
-          throw ioe;
-        }
-      } finally {
-        if (blockReader == null) {
-          IOUtils.cleanup(LOG, peer);
-        }
-      }
-    }
-  }
-
-  public static class BlockReaderPeer {
-    final Peer peer;
-    final boolean fromCache;
-    
-    BlockReaderPeer(Peer peer, boolean fromCache) {
-      this.peer = peer;
-      this.fromCache = fromCache;
-    }
-  }
-
-  /**
-   * Get the next DomainPeer-- either from the cache or by creating it.
-   *
-   * @return the next DomainPeer, or null if we could not construct one.
-   */
-  private BlockReaderPeer nextDomainPeer() {
-    if (remainingCacheTries > 0) {
-      Peer peer = clientContext.getPeerCache().get(datanode, true);
-      if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextDomainPeer: reusing existing peer " + peer);
-        }
-        return new BlockReaderPeer(peer, true);
-      }
-    }
-    DomainSocket sock = clientContext.getDomainSocketFactory().
-        createSocket(pathInfo, conf.getSocketTimeout());
-    if (sock == null) return null;
-    return new BlockReaderPeer(new DomainPeer(sock), false);
-  }
-
-  /**
-   * Get the next TCP-based peer-- either from the cache or by creating it.
-   *
-   * @return the next Peer, or null if we could not construct one.
-   *
-   * @throws IOException  If there was an error while constructing the peer
-   *                      (such as an InvalidEncryptionKeyException)
-   */
-  private BlockReaderPeer nextTcpPeer() throws IOException {
-    if (remainingCacheTries > 0) {
-      Peer peer = clientContext.getPeerCache().get(datanode, false);
-      if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextTcpPeer: reusing existing peer " + peer);
-        }
-        return new BlockReaderPeer(peer, true);
-      }
-    }
-    try {
-      Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
-        datanode);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("nextTcpPeer: created newConnectedPeer " + peer);
-      }
-      return new BlockReaderPeer(peer, false);
-    } catch (IOException e) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("nextTcpPeer: failed to create newConnectedPeer " +
-                  "connected to " + datanode);
-      }
-      throw e;
-    }
-  }
-
-  /**
-   * Determine if an exception is security-related.
-   *
-   * We need to handle these exceptions differently than other IOExceptions.
-   * They don't indicate a communication problem.  Instead, they mean that there
-   * is some action the client needs to take, such as refetching block tokens,
-   * renewing encryption keys, etc.
-   *
-   * @param ioe    The exception
-   * @return       True only if the exception is security-related.
-   */
-  private static boolean isSecurityException(IOException ioe) {
-    return (ioe instanceof InvalidToken) ||
-            (ioe instanceof InvalidEncryptionKeyException) ||
-            (ioe instanceof InvalidBlockTokenException) ||
-            (ioe instanceof AccessControlException);
-  }
-
-  @SuppressWarnings("deprecation")
-  private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
-    if (conf.getShortCircuitConf().isUseLegacyBlockReader()) {
-      return RemoteBlockReader.newBlockReader(fileName,
-          block, token, startOffset, length, conf.getIoBufferSize(),
-          verifyChecksum, clientName, peer, datanode,
-          clientContext.getPeerCache(), cachingStrategy);
-    } else {
-      return RemoteBlockReader2.newBlockReader(
-          fileName, block, token, startOffset, length,
-          verifyChecksum, clientName, peer, datanode,
-          clientContext.getPeerCache(), cachingStrategy);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "BlockReaderFactory(fileName=" + fileName + ", block=" + block + ")";
-  }
-
-  /**
-   * File name to print when accessing a block directly (from servlets)
-   * @param s Address of the block location
-   * @param poolId Block pool ID of the block
-   * @param blockId Block ID of the block
-   * @return string that has a file name for debug purposes
-   */
-  public static String getFileName(final InetSocketAddress s,
-      final String poolId, final long blockId) {
-    return s.toString() + ":" + poolId + ":" + blockId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
deleted file mode 100644
index cac5366..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
+++ /dev/null
@@ -1,369 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.BlockStorageLocation;
-import org.apache.hadoop.fs.HdfsVolumeId;
-import org.apache.hadoop.fs.VolumeId;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.token.Token;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class BlockStorageLocationUtil {
-  
-  static final Log LOG = LogFactory
-      .getLog(BlockStorageLocationUtil.class);
-  
-  /**
-   * Create a list of {@link VolumeBlockLocationCallable} corresponding to a set
-   * of datanodes and blocks. The blocks must all correspond to the same
-   * block pool.
-   * 
-   * @param datanodeBlocks
-   *          Map of datanodes to block replicas at each datanode
-   * @return callables Used to query each datanode for location information on
-   *         the block replicas at the datanode
-   */
-  private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables(
-      Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
-      int timeout, boolean connectToDnViaHostname, Span parent) {
-    
-    if (datanodeBlocks.isEmpty()) {
-      return Lists.newArrayList();
-    }
-    
-    // Construct the callables, one per datanode
-    List<VolumeBlockLocationCallable> callables = 
-        new ArrayList<VolumeBlockLocationCallable>();
-    for (Map.Entry<DatanodeInfo, List<LocatedBlock>> entry : datanodeBlocks
-        .entrySet()) {
-      // Construct RPC parameters
-      DatanodeInfo datanode = entry.getKey();
-      List<LocatedBlock> locatedBlocks = entry.getValue();
-      if (locatedBlocks.isEmpty()) {
-        continue;
-      }
-      
-      // Ensure that the blocks all are from the same block pool.
-      String poolId = locatedBlocks.get(0).getBlock().getBlockPoolId();
-      for (LocatedBlock lb : locatedBlocks) {
-        if (!poolId.equals(lb.getBlock().getBlockPoolId())) {
-          throw new IllegalArgumentException(
-              "All blocks to be queried must be in the same block pool: " +
-              locatedBlocks.get(0).getBlock() + " and " + lb +
-              " are from different pools.");
-        }
-      }
-      
-      long[] blockIds = new long[locatedBlocks.size()];
-      int i = 0;
-      List<Token<BlockTokenIdentifier>> dnTokens = 
-          new ArrayList<Token<BlockTokenIdentifier>>(
-          locatedBlocks.size());
-      for (LocatedBlock b : locatedBlocks) {
-        blockIds[i++] = b.getBlock().getBlockId();
-        dnTokens.add(b.getBlockToken());
-      }
-      VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable(
-          conf, datanode, poolId, blockIds, dnTokens, timeout, 
-          connectToDnViaHostname, parent);
-      callables.add(callable);
-    }
-    return callables;
-  }
-  
-  /**
-   * Queries datanodes for the blocks specified in <code>datanodeBlocks</code>,
-   * making one RPC to each datanode. These RPCs are made in parallel using a
-   * threadpool.
-   * 
-   * @param datanodeBlocks
-   *          Map of datanodes to the blocks present on the DN
-   * @return metadatas Map of datanodes to block metadata of the DN
-   * @throws InvalidBlockTokenException
-   *           if client does not have read access on a requested block
-   */
-  static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
-      Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
-      int poolsize, int timeoutMs, boolean connectToDnViaHostname)
-        throws InvalidBlockTokenException {
-
-    List<VolumeBlockLocationCallable> callables = 
-        createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs, 
-            connectToDnViaHostname, Trace.currentSpan());
-    
-    // Use a thread pool to execute the Callables in parallel
-    List<Future<HdfsBlocksMetadata>> futures = 
-        new ArrayList<Future<HdfsBlocksMetadata>>();
-    ExecutorService executor = new ScheduledThreadPoolExecutor(poolsize);
-    try {
-      futures = executor.invokeAll(callables, timeoutMs,
-          TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {
-      // Swallow the exception here, because we can return partial results
-    }
-    executor.shutdown();
-    
-    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas =
-        Maps.newHashMapWithExpectedSize(datanodeBlocks.size());
-    // Fill in metadatas with results from DN RPCs, where possible
-    for (int i = 0; i < futures.size(); i++) {
-      VolumeBlockLocationCallable callable = callables.get(i);
-      DatanodeInfo datanode = callable.getDatanodeInfo();
-      Future<HdfsBlocksMetadata> future = futures.get(i);
-      try {
-        HdfsBlocksMetadata metadata = future.get();
-        metadatas.put(callable.getDatanodeInfo(), metadata);
-      } catch (CancellationException e) {
-        LOG.info("Cancelled while waiting for datanode "
-            + datanode.getIpcAddr(false) + ": " + e.toString());
-      } catch (ExecutionException e) {
-        Throwable t = e.getCause();
-        if (t instanceof InvalidBlockTokenException) {
-          LOG.warn("Invalid access token when trying to retrieve "
-              + "information from datanode " + datanode.getIpcAddr(false));
-          throw (InvalidBlockTokenException) t;
-        }
-        else if (t instanceof UnsupportedOperationException) {
-          LOG.info("Datanode " + datanode.getIpcAddr(false) + " does not support"
-              + " required #getHdfsBlocksMetadata() API");
-          throw (UnsupportedOperationException) t;
-        } else {
-          LOG.info("Failed to query block locations on datanode " +
-              datanode.getIpcAddr(false) + ": " + t);
-        }
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Could not fetch information from datanode", t);
-        }
-      } catch (InterruptedException e) {
-        // Shouldn't happen, because invokeAll waits for all Futures to be ready
-        LOG.info("Interrupted while fetching HdfsBlocksMetadata");
-      }
-    }
-    
-    return metadatas;
-  }
-  
-  /**
-   * Group the per-replica {@link VolumeId} info returned from
-   * {@link DFSClient#queryDatanodesForHdfsBlocksMetadata(Map)} to be
-   * associated
-   * with the corresponding {@link LocatedBlock}.
-   * 
-   * @param blocks
-   *          Original LocatedBlock array
-   * @param metadatas
-   *          VolumeId information for the replicas on each datanode
-   * @return blockVolumeIds per-replica VolumeId information associated with the
-   *         parent LocatedBlock
-   */
-  static Map<LocatedBlock, List<VolumeId>> associateVolumeIdsWithBlocks(
-      List<LocatedBlock> blocks,
-      Map<DatanodeInfo, HdfsBlocksMetadata> metadatas) {
-    
-    // Initialize mapping of ExtendedBlock to LocatedBlock. 
-    // Used to associate results from DN RPCs to the parent LocatedBlock
-    Map<Long, LocatedBlock> blockIdToLocBlock = 
-        new HashMap<Long, LocatedBlock>();
-    for (LocatedBlock b : blocks) {
-      blockIdToLocBlock.put(b.getBlock().getBlockId(), b);
-    }
-    
-    // Initialize the mapping of blocks -> list of VolumeIds, one per replica
-    // This is filled out with real values from the DN RPCs
-    Map<LocatedBlock, List<VolumeId>> blockVolumeIds = 
-        new HashMap<LocatedBlock, List<VolumeId>>();
-    for (LocatedBlock b : blocks) {
-      ArrayList<VolumeId> l = new ArrayList<VolumeId>(b.getLocations().length);
-      for (int i = 0; i < b.getLocations().length; i++) {
-        l.add(null);
-      }
-      blockVolumeIds.put(b, l);
-    }
-    
-    // Iterate through the list of metadatas (one per datanode). 
-    // For each metadata, if it's valid, insert its volume location information 
-    // into the Map returned to the caller 
-    for (Map.Entry<DatanodeInfo, HdfsBlocksMetadata> entry : metadatas.entrySet()) {
-      DatanodeInfo datanode = entry.getKey();
-      HdfsBlocksMetadata metadata = entry.getValue();
-      // Check if metadata is valid
-      if (metadata == null) {
-        continue;
-      }
-      long[] metaBlockIds = metadata.getBlockIds();
-      List<byte[]> metaVolumeIds = metadata.getVolumeIds();
-      List<Integer> metaVolumeIndexes = metadata.getVolumeIndexes();
-      // Add VolumeId for each replica in the HdfsBlocksMetadata
-      for (int j = 0; j < metaBlockIds.length; j++) {
-        int volumeIndex = metaVolumeIndexes.get(j);
-        long blockId = metaBlockIds[j];
-        // Skip if block wasn't found, or not a valid index into metaVolumeIds
-        // Also skip if the DN responded with a block we didn't ask for
-        if (volumeIndex == Integer.MAX_VALUE
-            || volumeIndex >= metaVolumeIds.size()
-            || !blockIdToLocBlock.containsKey(blockId)) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("No data for block " + blockId);
-          }
-          continue;
-        }
-        // Get the VolumeId by indexing into the list of VolumeIds
-        // provided by the datanode
-        byte[] volumeId = metaVolumeIds.get(volumeIndex);
-        HdfsVolumeId id = new HdfsVolumeId(volumeId);
-        // Find out which index we are in the LocatedBlock's replicas
-        LocatedBlock locBlock = blockIdToLocBlock.get(blockId);
-        DatanodeInfo[] dnInfos = locBlock.getLocations();
-        int index = -1;
-        for (int k = 0; k < dnInfos.length; k++) {
-          if (dnInfos[k].equals(datanode)) {
-            index = k;
-            break;
-          }
-        }
-        if (index < 0) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Datanode responded with a block volume id we did" +
-                " not request, omitting.");
-          }
-          continue;
-        }
-        // Place VolumeId at the same index as the DN's index in the list of
-        // replicas
-        List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
-        volumeIds.set(index, id);
-      }
-    }
-    return blockVolumeIds;
-  }
-
-  /**
-   * Helper method to combine a list of {@link LocatedBlock} with associated
-   * {@link VolumeId} information to form a list of {@link BlockStorageLocation}
-   * .
-   */
-  static BlockStorageLocation[] convertToVolumeBlockLocations(
-      List<LocatedBlock> blocks, 
-      Map<LocatedBlock, List<VolumeId>> blockVolumeIds) throws IOException {
-    // Construct the final return value of VolumeBlockLocation[]
-    BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks);
-    List<BlockStorageLocation> volumeBlockLocs = 
-        new ArrayList<BlockStorageLocation>(locations.length);
-    for (int i = 0; i < locations.length; i++) {
-      LocatedBlock locBlock = blocks.get(i);
-      List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
-      BlockStorageLocation bsLoc = new BlockStorageLocation(locations[i], 
-          volumeIds.toArray(new VolumeId[0]));
-      volumeBlockLocs.add(bsLoc);
-    }
-    return volumeBlockLocs.toArray(new BlockStorageLocation[] {});
-  }
-  
-  /**
-   * Callable that sets up an RPC proxy to a datanode and queries it for
-   * volume location information for a list of ExtendedBlocks. 
-   */
-  private static class VolumeBlockLocationCallable implements 
-    Callable<HdfsBlocksMetadata> {
-    
-    private final Configuration configuration;
-    private final int timeout;
-    private final DatanodeInfo datanode;
-    private final String poolId;
-    private final long[] blockIds;
-    private final List<Token<BlockTokenIdentifier>> dnTokens;
-    private final boolean connectToDnViaHostname;
-    private final Span parentSpan;
-    
-    VolumeBlockLocationCallable(Configuration configuration,
-        DatanodeInfo datanode, String poolId, long []blockIds,
-        List<Token<BlockTokenIdentifier>> dnTokens, int timeout, 
-        boolean connectToDnViaHostname, Span parentSpan) {
-      this.configuration = configuration;
-      this.timeout = timeout;
-      this.datanode = datanode;
-      this.poolId = poolId;
-      this.blockIds = blockIds;
-      this.dnTokens = dnTokens;
-      this.connectToDnViaHostname = connectToDnViaHostname;
-      this.parentSpan = parentSpan;
-    }
-    
-    public DatanodeInfo getDatanodeInfo() {
-      return datanode;
-    }
-
-    @Override
-    public HdfsBlocksMetadata call() throws Exception {
-      HdfsBlocksMetadata metadata = null;
-      // Create the RPC proxy and make the RPC
-      ClientDatanodeProtocol cdp = null;
-      TraceScope scope =
-          Trace.startSpan("getHdfsBlocksMetadata", parentSpan);
-      try {
-        cdp = DFSUtilClient.createClientDatanodeProtocolProxy(
-            datanode, configuration,
-            timeout, connectToDnViaHostname);
-        metadata = cdp.getHdfsBlocksMetadata(poolId, blockIds, dnTokens);
-      } catch (IOException e) {
-        // Bubble this up to the caller, handle with the Future
-        throw e;
-      } finally {
-        scope.close();
-        if (cdp != null) {
-          RPC.stopProxy(cdp);
-        }
-      }
-      return metadata;
-    }
-  }
-}


[08/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
new file mode 100644
index 0000000..fb57825
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -0,0 +1,1904 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.NullScope;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceInfo;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/*********************************************************************
+ *
+ * The DataStreamer class is responsible for sending data packets to the
+ * datanodes in the pipeline. It retrieves a new blockid and block locations
+ * from the namenode, and starts streaming packets to the pipeline of
+ * Datanodes. Every packet has a sequence number associated with
+ * it. When all the packets for a block are sent out and acks for each
+ * if them are received, the DataStreamer closes the current block.
+ *
+ * The DataStreamer thread picks up packets from the dataQueue, sends it to
+ * the first datanode in the pipeline and moves it from the dataQueue to the
+ * ackQueue. The ResponseProcessor receives acks from the datanodes. When an
+ * successful ack for a packet is received from all datanodes, the
+ * ResponseProcessor removes the corresponding packet from the ackQueue.
+ *
+ * In case of error, all outstanding packets are moved from ackQueue. A new
+ * pipeline is setup by eliminating the bad datanode from the original
+ * pipeline. The DataStreamer now starts sending packets from the dataQueue.
+ *
+ *********************************************************************/
+
+@InterfaceAudience.Private
+class DataStreamer extends Daemon {
+  static final Logger LOG = LoggerFactory.getLogger(DataStreamer.class);
+
+  /**
+   * Create a socket for a write pipeline
+   *
+   * @param first the first datanode
+   * @param length the pipeline length
+   * @param client client
+   * @return the socket connected to the first datanode
+   */
+  static Socket createSocketForPipeline(final DatanodeInfo first,
+      final int length, final DFSClient client) throws IOException {
+    final DfsClientConf conf = client.getConf();
+    final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to datanode " + dnAddr);
+    }
+    final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
+    final Socket sock = client.socketFactory.createSocket();
+    final int timeout = client.getDatanodeReadTimeout(length);
+    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
+    sock.setSoTimeout(timeout);
+    sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Send buf size " + sock.getSendBufferSize());
+    }
+    return sock;
+  }
+
+  /**
+   * if this file is lazy persist
+   *
+   * @param stat the HdfsFileStatus of a file
+   * @return if this file is lazy persist
+   */
+  static boolean isLazyPersist(HdfsFileStatus stat) {
+    return stat.getStoragePolicy() == HdfsConstants.MEMORY_STORAGE_POLICY_ID;
+  }
+
+  /**
+   * release a list of packets to ByteArrayManager
+   *
+   * @param packets packets to be release
+   * @param bam ByteArrayManager
+   */
+  private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam) {
+    for(DFSPacket p : packets) {
+      p.releaseBuffer(bam);
+    }
+    packets.clear();
+  }
+  
+  static class LastExceptionInStreamer {
+    private IOException thrown;
+
+    synchronized void set(Throwable t) {
+      assert t != null;
+      this.thrown = t instanceof IOException ?
+          (IOException) t : new IOException(t);
+    }
+
+    synchronized void clear() {
+      thrown = null;
+    }
+
+    /** Check if there already is an exception. */
+    synchronized void check(boolean resetToNull) throws IOException {
+      if (thrown != null) {
+        if (LOG.isTraceEnabled()) {
+          // wrap and print the exception to know when the check is called
+          LOG.trace("Got Exception while checking", new Throwable(thrown));
+        }
+        final IOException e = thrown;
+        if (resetToNull) {
+          thrown = null;
+        }
+        throw e;
+      }
+    }
+
+    synchronized void throwException4Close() throws IOException {
+      check(false);
+      throw new ClosedChannelException();
+    }
+  }
+
+  static class ErrorState {
+    private boolean error = false;
+    private int badNodeIndex = -1;
+    private int restartingNodeIndex = -1;
+    private long restartingNodeDeadline = 0;
+    private final long datanodeRestartTimeout;
+
+    ErrorState(long datanodeRestartTimeout) {
+      this.datanodeRestartTimeout = datanodeRestartTimeout;
+    }
+
+    synchronized void reset() {
+      error = false;
+      badNodeIndex = -1;
+      restartingNodeIndex = -1;
+      restartingNodeDeadline = 0;
+    }
+
+    synchronized boolean hasError() {
+      return error;
+    }
+
+    synchronized boolean hasDatanodeError() {
+      return error && isNodeMarked();
+    }
+
+    synchronized void setError(boolean err) {
+      this.error = err;
+    }
+
+    synchronized void setBadNodeIndex(int index) {
+      this.badNodeIndex = index;
+    }
+
+    synchronized int getBadNodeIndex() {
+      return badNodeIndex;
+    }
+
+    synchronized int getRestartingNodeIndex() {
+      return restartingNodeIndex;
+    }
+
+    synchronized void initRestartingNode(int i, String message) {
+      restartingNodeIndex = i;
+      restartingNodeDeadline =  Time.monotonicNow() + datanodeRestartTimeout;
+      // If the data streamer has already set the primary node
+      // bad, clear it. It is likely that the write failed due to
+      // the DN shutdown. Even if it was a real failure, the pipeline
+      // recovery will take care of it.
+      badNodeIndex = -1;
+      LOG.info(message);
+    }
+
+    synchronized boolean isRestartingNode() {
+      return restartingNodeIndex >= 0;
+    }
+
+    synchronized boolean isNodeMarked() {
+      return badNodeIndex >= 0 || isRestartingNode();
+    }
+
+    /**
+     * This method is used when no explicit error report was received, but
+     * something failed. The first node is a suspect or unsure about the cause
+     * so that it is marked as failed.
+     */
+    synchronized void markFirstNodeIfNotMarked() {
+      // There should be no existing error and no ongoing restart.
+      if (!isNodeMarked()) {
+        badNodeIndex = 0;
+      }
+    }
+
+    synchronized void adjustState4RestartingNode() {
+      // Just took care of a node error while waiting for a node restart
+      if (restartingNodeIndex >= 0) {
+        // If the error came from a node further away than the restarting
+        // node, the restart must have been complete.
+        if (badNodeIndex > restartingNodeIndex) {
+          restartingNodeIndex = -1;
+        } else if (badNodeIndex < restartingNodeIndex) {
+          // the node index has shifted.
+          restartingNodeIndex--;
+        } else {
+          throw new IllegalStateException("badNodeIndex = " + badNodeIndex
+              + " = restartingNodeIndex = " + restartingNodeIndex);
+        }
+      }
+
+      if (!isRestartingNode()) {
+        error = false;
+      }
+      badNodeIndex = -1;
+    }
+
+    synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
+      if (restartingNodeIndex >= 0) {
+        if (!error) {
+          throw new IllegalStateException("error=false while checking" +
+              " restarting node deadline");
+        }
+
+        // check badNodeIndex
+        if (badNodeIndex == restartingNodeIndex) {
+          // ignore, if came from the restarting node
+          badNodeIndex = -1;
+        }
+        // not within the deadline
+        if (Time.monotonicNow() >= restartingNodeDeadline) {
+          // expired. declare the restarting node dead
+          restartingNodeDeadline = 0;
+          final int i = restartingNodeIndex;
+          restartingNodeIndex = -1;
+          LOG.warn("Datanode " + i + " did not restart within "
+              + datanodeRestartTimeout + "ms: " + nodes[i]);
+          // Mark the restarting node as failed. If there is any other failed
+          // node during the last pipeline construction attempt, it will not be
+          // overwritten/dropped. In this case, the restarting node will get
+          // excluded in the following attempt, if it still does not come up.
+          if (badNodeIndex == -1) {
+            badNodeIndex = i;
+          }
+        }
+      }
+    }
+  }
+
+  private volatile boolean streamerClosed = false;
+  private ExtendedBlock block; // its length is number of bytes acked
+  private Token<BlockTokenIdentifier> accessToken;
+  private DataOutputStream blockStream;
+  private DataInputStream blockReplyStream;
+  private ResponseProcessor response = null;
+  private volatile DatanodeInfo[] nodes = null; // list of targets for current block
+  private volatile StorageType[] storageTypes = null;
+  private volatile String[] storageIDs = null;
+  private final ErrorState errorState;
+
+  private BlockConstructionStage stage;  // block construction stage
+  private long bytesSent = 0; // number of bytes that've been sent
+  private final boolean isLazyPersistFile;
+
+  /** Nodes have been used in the pipeline before and have failed. */
+  private final List<DatanodeInfo> failed = new ArrayList<>();
+  /** The last ack sequence number before pipeline failure. */
+  private long lastAckedSeqnoBeforeFailure = -1;
+  private int pipelineRecoveryCount = 0;
+  /** Has the current block been hflushed? */
+  private boolean isHflushed = false;
+  /** Append on an existing block? */
+  private final boolean isAppend;
+
+  private long currentSeqno = 0;
+  private long lastQueuedSeqno = -1;
+  private long lastAckedSeqno = -1;
+  private long bytesCurBlock = 0; // bytes written in current block
+  private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
+  private Socket s;
+
+  private final DFSClient dfsClient;
+  private final String src;
+  /** Only for DataTransferProtocol.writeBlock(..) */
+  private final DataChecksum checksum4WriteBlock;
+  private final Progressable progress;
+  private final HdfsFileStatus stat;
+  // appending to existing partial block
+  private volatile boolean appendChunk = false;
+  // both dataQueue and ackQueue are protected by dataQueue lock
+  private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
+  private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
+  private final AtomicReference<CachingStrategy> cachingStrategy;
+  private final ByteArrayManager byteArrayManager;
+  //persist blocks on namenode
+  private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
+  private boolean failPacket = false;
+  private final long dfsclientSlowLogThresholdMs;
+  private long artificialSlowdown = 0;
+  // List of congested data nodes. The stream will back off if the DataNodes
+  // are congested
+  private final List<DatanodeInfo> congestedNodes = new ArrayList<>();
+  private static final int CONGESTION_BACKOFF_MEAN_TIME_IN_MS = 5000;
+  private static final int CONGESTION_BACK_OFF_MAX_TIME_IN_MS =
+      CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
+  private int lastCongestionBackoffTime;
+
+  private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+  private final String[] favoredNodes;
+
+  private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
+                       Progressable progress, DataChecksum checksum,
+                       AtomicReference<CachingStrategy> cachingStrategy,
+                       ByteArrayManager byteArrayManage,
+                       boolean isAppend, String[] favoredNodes) {
+    this.dfsClient = dfsClient;
+    this.src = src;
+    this.progress = progress;
+    this.stat = stat;
+    this.checksum4WriteBlock = checksum;
+    this.cachingStrategy = cachingStrategy;
+    this.byteArrayManager = byteArrayManage;
+    this.isLazyPersistFile = isLazyPersist(stat);
+    this.isAppend = isAppend;
+    this.favoredNodes = favoredNodes;
+
+    final DfsClientConf conf = dfsClient.getConf();
+    this.dfsclientSlowLogThresholdMs = conf.getSlowIoWarningThresholdMs();
+    this.excludedNodes = initExcludedNodes(conf.getExcludedNodesCacheExpiry());
+    this.errorState = new ErrorState(conf.getDatanodeRestartTimeout());
+  }
+
+  /**
+   * construction with tracing info
+   */
+  DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
+               String src, Progressable progress, DataChecksum checksum,
+               AtomicReference<CachingStrategy> cachingStrategy,
+               ByteArrayManager byteArrayManage, String[] favoredNodes) {
+    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage, false, favoredNodes);
+    this.block = block;
+    stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+  }
+
+  /**
+   * Construct a data streamer for appending to the last partial block
+   * @param lastBlock last block of the file to be appended
+   * @param stat status of the file to be appended
+   * @throws IOException if error occurs
+   */
+  DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
+               String src, Progressable progress, DataChecksum checksum,
+               AtomicReference<CachingStrategy> cachingStrategy,
+               ByteArrayManager byteArrayManage) throws IOException {
+    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage, true, null);
+    stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
+    block = lastBlock.getBlock();
+    bytesSent = block.getNumBytes();
+    accessToken = lastBlock.getBlockToken();
+  }
+
+  /**
+   * Set pipeline in construction
+   *
+   * @param lastBlock the last block of a file
+   * @throws IOException
+   */
+  void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{
+    // setup pipeline to append to the last block XXX retries??
+    setPipeline(lastBlock);
+    if (nodes.length < 1) {
+      throw new IOException("Unable to retrieve blocks locations " +
+          " for last block " + block +
+          "of file " + src);
+    }
+  }
+
+  private void setPipeline(LocatedBlock lb) {
+    setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
+  }
+
+  private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
+                           String[] storageIDs) {
+    this.nodes = nodes;
+    this.storageTypes = storageTypes;
+    this.storageIDs = storageIDs;
+  }
+
+  /**
+   * Initialize for data streaming
+   */
+  private void initDataStreaming() {
+    this.setName("DataStreamer for file " + src +
+        " block " + block);
+    response = new ResponseProcessor(nodes);
+    response.start();
+    stage = BlockConstructionStage.DATA_STREAMING;
+  }
+
+  private void endBlock() {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Closing old block " + block);
+    }
+    this.setName("DataStreamer for file " + src);
+    closeResponder();
+    closeStream();
+    setPipeline(null, null, null);
+    stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+  }
+
+  private boolean shouldStop() {
+    return streamerClosed || errorState.hasError() || !dfsClient.clientRunning;
+  }
+
+  /*
+   * streamer thread is the only thread that opens streams to datanode,
+   * and closes them. Any error recovery is also done by this thread.
+   */
+  @Override
+  public void run() {
+    long lastPacket = Time.monotonicNow();
+    TraceScope scope = NullScope.INSTANCE;
+    while (!streamerClosed && dfsClient.clientRunning) {
+      // if the Responder encountered an error, shutdown Responder
+      if (errorState.hasError() && response != null) {
+        try {
+          response.close();
+          response.join();
+          response = null;
+        } catch (InterruptedException  e) {
+          LOG.warn("Caught exception", e);
+        }
+      }
+
+      DFSPacket one;
+      try {
+        // process datanode IO errors if any
+        boolean doSleep = processDatanodeError();
+
+        final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
+        synchronized (dataQueue) {
+          // wait for a packet to be sent.
+          long now = Time.monotonicNow();
+          while ((!shouldStop() && dataQueue.size() == 0 &&
+              (stage != BlockConstructionStage.DATA_STREAMING ||
+                  stage == BlockConstructionStage.DATA_STREAMING &&
+                      now - lastPacket < halfSocketTimeout)) || doSleep ) {
+            long timeout = halfSocketTimeout - (now-lastPacket);
+            timeout = timeout <= 0 ? 1000 : timeout;
+            timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
+                timeout : 1000;
+            try {
+              dataQueue.wait(timeout);
+            } catch (InterruptedException  e) {
+              LOG.warn("Caught exception", e);
+            }
+            doSleep = false;
+            now = Time.monotonicNow();
+          }
+          if (shouldStop()) {
+            continue;
+          }
+          // get packet to be sent.
+          if (dataQueue.isEmpty()) {
+            one = createHeartbeatPacket();
+          } else {
+            try {
+              backOffIfNecessary();
+            } catch (InterruptedException e) {
+              LOG.warn("Caught exception", e);
+            }
+            one = dataQueue.getFirst(); // regular data packet
+            long parents[] = one.getTraceParents();
+            if (parents.length > 0) {
+              scope = Trace.startSpan("dataStreamer", new TraceInfo(0, parents[0]));
+              // TODO: use setParents API once it's available from HTrace 3.2
+              // scope = Trace.startSpan("dataStreamer", Sampler.ALWAYS);
+              // scope.getSpan().setParents(parents);
+            }
+          }
+        }
+
+        // get new block from namenode.
+        if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Allocating new block");
+          }
+          setPipeline(nextBlockOutputStream());
+          initDataStreaming();
+        } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Append to block " + block);
+          }
+          setupPipelineForAppendOrRecovery();
+          if (streamerClosed) {
+            continue;
+          }
+          initDataStreaming();
+        }
+
+        long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
+        if (lastByteOffsetInBlock > stat.getBlockSize()) {
+          throw new IOException("BlockSize " + stat.getBlockSize() +
+              " is smaller than data size. " +
+              " Offset of packet in block " +
+              lastByteOffsetInBlock +
+              " Aborting file " + src);
+        }
+
+        if (one.isLastPacketInBlock()) {
+          // wait for all data packets have been successfully acked
+          synchronized (dataQueue) {
+            while (!shouldStop() && ackQueue.size() != 0) {
+              try {
+                // wait for acks to arrive from datanodes
+                dataQueue.wait(1000);
+              } catch (InterruptedException  e) {
+                LOG.warn("Caught exception", e);
+              }
+            }
+          }
+          if (shouldStop()) {
+            continue;
+          }
+          stage = BlockConstructionStage.PIPELINE_CLOSE;
+        }
+
+        // send the packet
+        Span span = null;
+        synchronized (dataQueue) {
+          // move packet from dataQueue to ackQueue
+          if (!one.isHeartbeatPacket()) {
+            span = scope.detach();
+            one.setTraceSpan(span);
+            dataQueue.removeFirst();
+            ackQueue.addLast(one);
+            dataQueue.notifyAll();
+          }
+        }
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("DataStreamer block " + block +
+              " sending packet " + one);
+        }
+
+        // write out data to remote datanode
+        TraceScope writeScope = Trace.startSpan("writeTo", span);
+        try {
+          one.writeTo(blockStream);
+          blockStream.flush();
+        } catch (IOException e) {
+          // HDFS-3398 treat primary DN is down since client is unable to
+          // write to primary DN. If a failed or restarting node has already
+          // been recorded by the responder, the following call will have no
+          // effect. Pipeline recovery can handle only one node error at a
+          // time. If the primary node fails again during the recovery, it
+          // will be taken out then.
+          errorState.markFirstNodeIfNotMarked();
+          throw e;
+        } finally {
+          writeScope.close();
+        }
+        lastPacket = Time.monotonicNow();
+
+        // update bytesSent
+        long tmpBytesSent = one.getLastByteOffsetBlock();
+        if (bytesSent < tmpBytesSent) {
+          bytesSent = tmpBytesSent;
+        }
+
+        if (shouldStop()) {
+          continue;
+        }
+
+        // Is this block full?
+        if (one.isLastPacketInBlock()) {
+          // wait for the close packet has been acked
+          synchronized (dataQueue) {
+            while (!shouldStop() && ackQueue.size() != 0) {
+              dataQueue.wait(1000);// wait for acks to arrive from datanodes
+            }
+          }
+          if (shouldStop()) {
+            continue;
+          }
+
+          endBlock();
+        }
+        if (progress != null) { progress.progress(); }
+
+        // This is used by unit test to trigger race conditions.
+        if (artificialSlowdown != 0 && dfsClient.clientRunning) {
+          Thread.sleep(artificialSlowdown);
+        }
+      } catch (Throwable e) {
+        // Log warning if there was a real error.
+        if (!errorState.isRestartingNode()) {
+          // Since their messages are descriptive enough, do not always
+          // log a verbose stack-trace WARN for quota exceptions.
+          if (e instanceof QuotaExceededException) {
+            LOG.debug("DataStreamer Quota Exception", e);
+          } else {
+            LOG.warn("DataStreamer Exception", e);
+          }
+        }
+        lastException.set(e);
+        assert !(e instanceof NullPointerException);
+        errorState.setError(true);
+        if (!errorState.isNodeMarked()) {
+          // Not a datanode issue
+          streamerClosed = true;
+        }
+      } finally {
+        scope.close();
+      }
+    }
+    closeInternal();
+  }
+
+  private void closeInternal() {
+    closeResponder();       // close and join
+    closeStream();
+    streamerClosed = true;
+    release();
+    synchronized (dataQueue) {
+      dataQueue.notifyAll();
+    }
+  }
+
+  /**
+   * release the DFSPackets in the two queues
+   *
+   */
+  void release() {
+    synchronized (dataQueue) {
+      releaseBuffer(dataQueue, byteArrayManager);
+      releaseBuffer(ackQueue, byteArrayManager);
+    }
+  }
+
+  /**
+   * wait for the ack of seqno
+   *
+   * @param seqno the sequence number to be acked
+   * @throws IOException
+   */
+  void waitForAckedSeqno(long seqno) throws IOException {
+    TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Waiting for ack for: " + seqno);
+      }
+      long begin = Time.monotonicNow();
+      try {
+        synchronized (dataQueue) {
+          while (!streamerClosed) {
+            checkClosed();
+            if (lastAckedSeqno >= seqno) {
+              break;
+            }
+            try {
+              dataQueue.wait(1000); // when we receive an ack, we notify on
+              // dataQueue
+            } catch (InterruptedException ie) {
+              throw new InterruptedIOException(
+                  "Interrupted while waiting for data to be acknowledged by pipeline");
+            }
+          }
+        }
+        checkClosed();
+      } catch (ClosedChannelException e) {
+      }
+      long duration = Time.monotonicNow() - begin;
+      if (duration > dfsclientSlowLogThresholdMs) {
+        LOG.warn("Slow waitForAckedSeqno took " + duration
+            + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
+      }
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * wait for space of dataQueue and queue the packet
+   *
+   * @param packet  the DFSPacket to be queued
+   * @throws IOException
+   */
+  void waitAndQueuePacket(DFSPacket packet) throws IOException {
+    synchronized (dataQueue) {
+      try {
+        // If queue is full, then wait till we have enough space
+        boolean firstWait = true;
+        try {
+          while (!streamerClosed && dataQueue.size() + ackQueue.size() >
+              dfsClient.getConf().getWriteMaxPackets()) {
+            if (firstWait) {
+              Span span = Trace.currentSpan();
+              if (span != null) {
+                span.addTimelineAnnotation("dataQueue.wait");
+              }
+              firstWait = false;
+            }
+            try {
+              dataQueue.wait();
+            } catch (InterruptedException e) {
+              // If we get interrupted while waiting to queue data, we still need to get rid
+              // of the current packet. This is because we have an invariant that if
+              // currentPacket gets full, it will get queued before the next writeChunk.
+              //
+              // Rather than wait around for space in the queue, we should instead try to
+              // return to the caller as soon as possible, even though we slightly overrun
+              // the MAX_PACKETS length.
+              Thread.currentThread().interrupt();
+              break;
+            }
+          }
+        } finally {
+          Span span = Trace.currentSpan();
+          if ((span != null) && (!firstWait)) {
+            span.addTimelineAnnotation("end.wait");
+          }
+        }
+        checkClosed();
+        queuePacket(packet);
+      } catch (ClosedChannelException e) {
+      }
+    }
+  }
+
+  /*
+   * close the streamer, should be called only by an external thread
+   * and only after all data to be sent has been flushed to datanode.
+   *
+   * Interrupt this data streamer if force is true
+   *
+   * @param force if this data stream is forced to be closed
+   */
+  void close(boolean force) {
+    streamerClosed = true;
+    synchronized (dataQueue) {
+      dataQueue.notifyAll();
+    }
+    if (force) {
+      this.interrupt();
+    }
+  }
+
+
+  private void checkClosed() throws IOException {
+    if (streamerClosed) {
+      lastException.throwException4Close();
+    }
+  }
+
+  private void closeResponder() {
+    if (response != null) {
+      try {
+        response.close();
+        response.join();
+      } catch (InterruptedException  e) {
+        LOG.warn("Caught exception", e);
+      } finally {
+        response = null;
+      }
+    }
+  }
+
+  private void closeStream() {
+    final MultipleIOException.Builder b = new MultipleIOException.Builder();
+
+    if (blockStream != null) {
+      try {
+        blockStream.close();
+      } catch (IOException e) {
+        b.add(e);
+      } finally {
+        blockStream = null;
+      }
+    }
+    if (blockReplyStream != null) {
+      try {
+        blockReplyStream.close();
+      } catch (IOException e) {
+        b.add(e);
+      } finally {
+        blockReplyStream = null;
+      }
+    }
+    if (null != s) {
+      try {
+        s.close();
+      } catch (IOException e) {
+        b.add(e);
+      } finally {
+        s = null;
+      }
+    }
+
+    final IOException ioe = b.build();
+    if (ioe != null) {
+      lastException.set(ioe);
+    }
+  }
+
+  /**
+   * Examine whether it is worth waiting for a node to restart.
+   * @param index the node index
+   */
+  boolean shouldWaitForRestart(int index) {
+    // Only one node in the pipeline.
+    if (nodes.length == 1) {
+      return true;
+    }
+
+    // Is it a local node?
+    InetAddress addr = null;
+    try {
+      addr = InetAddress.getByName(nodes[index].getIpAddr());
+    } catch (java.net.UnknownHostException e) {
+      // we are passing an ip address. this should not happen.
+      assert false;
+    }
+
+    if (addr != null && NetUtils.isLocalAddress(addr)) {
+      return true;
+    }
+    return false;
+  }
+
+  //
+  // Processes responses from the datanodes.  A packet is removed
+  // from the ackQueue when its response arrives.
+  //
+  private class ResponseProcessor extends Daemon {
+
+    private volatile boolean responderClosed = false;
+    private DatanodeInfo[] targets = null;
+    private boolean isLastPacketInBlock = false;
+
+    ResponseProcessor (DatanodeInfo[] targets) {
+      this.targets = targets;
+    }
+
+    @Override
+    public void run() {
+
+      setName("ResponseProcessor for block " + block);
+      PipelineAck ack = new PipelineAck();
+
+      TraceScope scope = NullScope.INSTANCE;
+      while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
+        // process responses from datanodes.
+        try {
+          // read an ack from the pipeline
+          long begin = Time.monotonicNow();
+          ack.readFields(blockReplyStream);
+          long duration = Time.monotonicNow() - begin;
+          if (duration > dfsclientSlowLogThresholdMs
+              && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
+            LOG.warn("Slow ReadProcessor read fields took " + duration
+                + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
+                + ack + ", targets: " + Arrays.asList(targets));
+          } else if (LOG.isDebugEnabled()) {
+            LOG.debug("DFSClient " + ack);
+          }
+
+          long seqno = ack.getSeqno();
+          // processes response status from datanodes.
+          ArrayList<DatanodeInfo> congestedNodesFromAck = new ArrayList<>();
+          for (int i = ack.getNumOfReplies()-1; i >=0  && dfsClient.clientRunning; i--) {
+            final Status reply = PipelineAck.getStatusFromHeader(ack
+                .getHeaderFlag(i));
+            if (PipelineAck.getECNFromHeader(ack.getHeaderFlag(i)) ==
+                PipelineAck.ECN.CONGESTED) {
+              congestedNodesFromAck.add(targets[i]);
+            }
+            // Restart will not be treated differently unless it is
+            // the local node or the only one in the pipeline.
+            if (PipelineAck.isRestartOOBStatus(reply) &&
+                shouldWaitForRestart(i)) {
+              final String message = "Datanode " + i + " is restarting: "
+                  + targets[i];
+              errorState.initRestartingNode(i, message);
+              throw new IOException(message);
+            }
+            // node error
+            if (reply != SUCCESS) {
+              errorState.setBadNodeIndex(i); // mark bad datanode
+              throw new IOException("Bad response " + reply +
+                  " for " + block + " from datanode " + targets[i]);
+            }
+          }
+
+          if (!congestedNodesFromAck.isEmpty()) {
+            synchronized (congestedNodes) {
+              congestedNodes.clear();
+              congestedNodes.addAll(congestedNodesFromAck);
+            }
+          } else {
+            synchronized (congestedNodes) {
+              congestedNodes.clear();
+              lastCongestionBackoffTime = 0;
+            }
+          }
+
+          assert seqno != PipelineAck.UNKOWN_SEQNO :
+              "Ack for unknown seqno should be a failed ack: " + ack;
+          if (seqno == DFSPacket.HEART_BEAT_SEQNO) {  // a heartbeat ack
+            continue;
+          }
+
+          // a success ack for a data packet
+          DFSPacket one;
+          synchronized (dataQueue) {
+            one = ackQueue.getFirst();
+          }
+          if (one.getSeqno() != seqno) {
+            throw new IOException("ResponseProcessor: Expecting seqno " +
+                " for block " + block +
+                one.getSeqno() + " but received " + seqno);
+          }
+          isLastPacketInBlock = one.isLastPacketInBlock();
+
+          // Fail the packet write for testing in order to force a
+          // pipeline recovery.
+          if (DFSClientFaultInjector.get().failPacket() &&
+              isLastPacketInBlock) {
+            failPacket = true;
+            throw new IOException(
+                "Failing the last packet for testing.");
+          }
+
+          // update bytesAcked
+          block.setNumBytes(one.getLastByteOffsetBlock());
+
+          synchronized (dataQueue) {
+            scope = Trace.continueSpan(one.getTraceSpan());
+            one.setTraceSpan(null);
+            lastAckedSeqno = seqno;
+            ackQueue.removeFirst();
+            dataQueue.notifyAll();
+
+            one.releaseBuffer(byteArrayManager);
+          }
+        } catch (Exception e) {
+          if (!responderClosed) {
+            lastException.set(e);
+            errorState.setError(true);
+            errorState.markFirstNodeIfNotMarked();
+            synchronized (dataQueue) {
+              dataQueue.notifyAll();
+            }
+            if (!errorState.isRestartingNode()) {
+              LOG.warn("Exception for " + block, e);
+            }
+            responderClosed = true;
+          }
+        } finally {
+            scope.close();
+        }
+      }
+    }
+
+    void close() {
+      responderClosed = true;
+      this.interrupt();
+    }
+  }
+
+  /**
+   * If this stream has encountered any errors, shutdown threads
+   * and mark the stream as closed.
+   *
+   * @return true if it should sleep for a while after returning.
+   */
+  private boolean processDatanodeError() throws IOException {
+    if (!errorState.hasDatanodeError()) {
+      return false;
+    }
+    if (response != null) {
+      LOG.info("Error Recovery for " + block +
+          " waiting for responder to exit. ");
+      return true;
+    }
+    closeStream();
+
+    // move packets from ack queue to front of the data queue
+    synchronized (dataQueue) {
+      dataQueue.addAll(0, ackQueue);
+      ackQueue.clear();
+    }
+
+    // Record the new pipeline failure recovery.
+    if (lastAckedSeqnoBeforeFailure != lastAckedSeqno) {
+      lastAckedSeqnoBeforeFailure = lastAckedSeqno;
+      pipelineRecoveryCount = 1;
+    } else {
+      // If we had to recover the pipeline five times in a row for the
+      // same packet, this client likely has corrupt data or corrupting
+      // during transmission.
+      if (++pipelineRecoveryCount > 5) {
+        LOG.warn("Error recovering pipeline for writing " +
+            block + ". Already retried 5 times for the same packet.");
+        lastException.set(new IOException("Failing write. Tried pipeline " +
+            "recovery 5 times without success."));
+        streamerClosed = true;
+        return false;
+      }
+    }
+    boolean doSleep = setupPipelineForAppendOrRecovery();
+
+    if (!streamerClosed && dfsClient.clientRunning) {
+      if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
+
+        // If we had an error while closing the pipeline, we go through a fast-path
+        // where the BlockReceiver does not run. Instead, the DataNode just finalizes
+        // the block immediately during the 'connect ack' process. So, we want to pull
+        // the end-of-block packet from the dataQueue, since we don't actually have
+        // a true pipeline to send it over.
+        //
+        // We also need to set lastAckedSeqno to the end-of-block Packet's seqno, so that
+        // a client waiting on close() will be aware that the flush finished.
+        synchronized (dataQueue) {
+          DFSPacket endOfBlockPacket = dataQueue.remove();  // remove the end of block packet
+          Span span = endOfBlockPacket.getTraceSpan();
+          if (span != null) {
+            // Close any trace span associated with this Packet
+            TraceScope scope = Trace.continueSpan(span);
+            scope.close();
+          }
+          assert endOfBlockPacket.isLastPacketInBlock();
+          assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
+          lastAckedSeqno = endOfBlockPacket.getSeqno();
+          dataQueue.notifyAll();
+        }
+        endBlock();
+      } else {
+        initDataStreaming();
+      }
+    }
+
+    return doSleep;
+  }
+
+  void setHflush() {
+    isHflushed = true;
+  }
+
+  private int findNewDatanode(final DatanodeInfo[] original
+  ) throws IOException {
+    if (nodes.length != original.length + 1) {
+      throw new IOException(
+          new StringBuilder()
+              .append("Failed to replace a bad datanode on the existing pipeline ")
+              .append("due to no more good datanodes being available to try. ")
+              .append("(Nodes: current=").append(Arrays.asList(nodes))
+              .append(", original=").append(Arrays.asList(original)).append("). ")
+              .append("The current failed datanode replacement policy is ")
+              .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
+              .append("a client may configure this via '")
+              .append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
+              .append("' in its configuration.")
+              .toString());
+    }
+    for(int i = 0; i < nodes.length; i++) {
+      int j = 0;
+      for(; j < original.length && !nodes[i].equals(original[j]); j++);
+      if (j == original.length) {
+        return i;
+      }
+    }
+    throw new IOException("Failed: new datanode not found: nodes="
+        + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
+  }
+
+  private void addDatanode2ExistingPipeline() throws IOException {
+    if (DataTransferProtocol.LOG.isDebugEnabled()) {
+      DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
+    }
+      /*
+       * Is data transfer necessary?  We have the following cases.
+       *
+       * Case 1: Failure in Pipeline Setup
+       * - Append
+       *    + Transfer the stored replica, which may be a RBW or a finalized.
+       * - Create
+       *    + If no data, then no transfer is required.
+       *    + If there are data written, transfer RBW. This case may happens
+       *      when there are streaming failure earlier in this pipeline.
+       *
+       * Case 2: Failure in Streaming
+       * - Append/Create:
+       *    + transfer RBW
+       *
+       * Case 3: Failure in Close
+       * - Append/Create:
+       *    + no transfer, let NameNode replicates the block.
+       */
+    if (!isAppend && lastAckedSeqno < 0
+        && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+      //no data have been written
+      return;
+    } else if (stage == BlockConstructionStage.PIPELINE_CLOSE
+        || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+      //pipeline is closing
+      return;
+    }
+
+    //get a new datanode
+    final DatanodeInfo[] original = nodes;
+    final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
+        src, stat.getFileId(), block, nodes, storageIDs,
+        failed.toArray(new DatanodeInfo[failed.size()]),
+        1, dfsClient.clientName);
+    setPipeline(lb);
+
+    //find the new datanode
+    final int d = findNewDatanode(original);
+
+    //transfer replica
+    final DatanodeInfo src = d == 0? nodes[1]: nodes[d - 1];
+    final DatanodeInfo[] targets = {nodes[d]};
+    final StorageType[] targetStorageTypes = {storageTypes[d]};
+    transfer(src, targets, targetStorageTypes, lb.getBlockToken());
+  }
+
+  private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
+                        final StorageType[] targetStorageTypes,
+                        final Token<BlockTokenIdentifier> blockToken) throws IOException {
+    //transfer replica to the new datanode
+    Socket sock = null;
+    DataOutputStream out = null;
+    DataInputStream in = null;
+    try {
+      sock = createSocketForPipeline(src, 2, dfsClient);
+      final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
+      final long readTimeout = dfsClient.getDatanodeReadTimeout(2);
+
+      OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
+      InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);
+      IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
+          unbufOut, unbufIn, dfsClient, blockToken, src);
+      unbufOut = saslStreams.out;
+      unbufIn = saslStreams.in;
+      out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+          DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
+      in = new DataInputStream(unbufIn);
+
+      //send the TRANSFER_BLOCK request
+      new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
+          targets, targetStorageTypes);
+      out.flush();
+
+      //ack
+      BlockOpResponseProto response =
+          BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+      if (SUCCESS != response.getStatus()) {
+        throw new IOException("Failed to add a datanode");
+      }
+    } finally {
+      IOUtils.closeStream(in);
+      IOUtils.closeStream(out);
+      IOUtils.closeSocket(sock);
+    }
+  }
+
+  /**
+   * Open a DataStreamer to a DataNode pipeline so that
+   * it can be written to.
+   * This happens when a file is appended or data streaming fails
+   * It keeps on trying until a pipeline is setup
+   */
+  private boolean setupPipelineForAppendOrRecovery() throws IOException {
+    // check number of datanodes
+    if (nodes == null || nodes.length == 0) {
+      String msg = "Could not get block locations. " + "Source file \""
+          + src + "\" - Aborting...";
+      LOG.warn(msg);
+      lastException.set(new IOException(msg));
+      streamerClosed = true;
+      return false;
+    }
+
+    boolean success = false;
+    long newGS = 0L;
+    while (!success && !streamerClosed && dfsClient.clientRunning) {
+      if (!handleRestartingDatanode()) {
+        return false;
+      }
+
+      final boolean isRecovery = errorState.hasError();
+      if (!handleBadDatanode()) {
+        return false;
+      }
+
+      handleDatanodeReplacement();
+
+      // get a new generation stamp and an access token
+      final LocatedBlock lb = updateBlockForPipeline();
+      newGS = lb.getBlock().getGenerationStamp();
+      accessToken = lb.getBlockToken();
+
+      // set up the pipeline again with the remaining nodes
+      success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
+
+      failPacket4Testing();
+
+      errorState.checkRestartingNodeDeadline(nodes);
+    } // while
+
+    if (success) {
+      block = updatePipeline(newGS);
+    }
+    return false; // do not sleep, continue processing
+  }
+
+  /**
+   * Sleep if a node is restarting.
+   * This process is repeated until the deadline or the node starts back up.
+   * @return true if it should continue.
+   */
+  private boolean handleRestartingDatanode() {
+    if (errorState.isRestartingNode()) {
+      // 4 seconds or the configured deadline period, whichever is shorter.
+      // This is the retry interval and recovery will be retried in this
+      // interval until timeout or success.
+      final long delay = Math.min(errorState.datanodeRestartTimeout, 4000L);
+      try {
+        Thread.sleep(delay);
+      } catch (InterruptedException ie) {
+        lastException.set(new IOException(
+            "Interrupted while waiting for restarting "
+            + nodes[errorState.getRestartingNodeIndex()]));
+        streamerClosed = true;
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Remove bad node from list of nodes if badNodeIndex was set.
+   * @return true if it should continue.
+   */
+  private boolean handleBadDatanode() {
+    final int badNodeIndex = errorState.getBadNodeIndex();
+    if (badNodeIndex >= 0) {
+      if (nodes.length <= 1) {
+        lastException.set(new IOException("All datanodes "
+            + Arrays.toString(nodes) + " are bad. Aborting..."));
+        streamerClosed = true;
+        return false;
+      }
+
+      LOG.warn("Error Recovery for " + block + " in pipeline "
+          + Arrays.toString(nodes) + ": datanode " + badNodeIndex
+          + "("+ nodes[badNodeIndex] + ") is bad.");
+      failed.add(nodes[badNodeIndex]);
+
+      DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
+      arraycopy(nodes, newnodes, badNodeIndex);
+
+      final StorageType[] newStorageTypes = new StorageType[newnodes.length];
+      arraycopy(storageTypes, newStorageTypes, badNodeIndex);
+
+      final String[] newStorageIDs = new String[newnodes.length];
+      arraycopy(storageIDs, newStorageIDs, badNodeIndex);
+
+      setPipeline(newnodes, newStorageTypes, newStorageIDs);
+
+      errorState.adjustState4RestartingNode();
+      lastException.clear();
+    }
+    return true;
+  }
+
+  /** Add a datanode if replace-datanode policy is satisfied. */
+  private void handleDatanodeReplacement() throws IOException {
+    if (dfsClient.dtpReplaceDatanodeOnFailure.satisfy(stat.getReplication(),
+        nodes, isAppend, isHflushed)) {
+      try {
+        addDatanode2ExistingPipeline();
+      } catch(IOException ioe) {
+        if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
+          throw ioe;
+        }
+        LOG.warn("Failed to replace datanode."
+            + " Continue with the remaining datanodes since "
+            + BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
+            + " is set to true.", ioe);
+      }
+    }
+  }
+
+  private void failPacket4Testing() {
+    if (failPacket) { // for testing
+      failPacket = false;
+      try {
+        // Give DNs time to send in bad reports. In real situations,
+        // good reports should follow bad ones, if client committed
+        // with those nodes.
+        Thread.sleep(2000);
+      } catch (InterruptedException ie) {}
+    }
+  }
+
+  LocatedBlock updateBlockForPipeline() throws IOException {
+    return dfsClient.namenode.updateBlockForPipeline(
+        block, dfsClient.clientName);
+  }
+
+  /** update pipeline at the namenode */
+  ExtendedBlock updatePipeline(long newGS) throws IOException {
+    final ExtendedBlock newBlock = new ExtendedBlock(
+        block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
+    dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+        nodes, storageIDs);
+    return newBlock;
+  }
+
+  /**
+   * Open a DataStreamer to a DataNode so that it can be written to.
+   * This happens when a file is created and each time a new block is allocated.
+   * Must get block ID and the IDs of the destinations from the namenode.
+   * Returns the list of target datanodes.
+   */
+  private LocatedBlock nextBlockOutputStream() throws IOException {
+    LocatedBlock lb = null;
+    DatanodeInfo[] nodes = null;
+    StorageType[] storageTypes = null;
+    int count = dfsClient.getConf().getNumBlockWriteRetry();
+    boolean success = false;
+    ExtendedBlock oldBlock = block;
+    do {
+      errorState.reset();
+      lastException.clear();
+      success = false;
+
+      DatanodeInfo[] excluded =
+          excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
+              .keySet()
+              .toArray(new DatanodeInfo[0]);
+      block = oldBlock;
+      lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
+      block = lb.getBlock();
+      block.setNumBytes(0);
+      bytesSent = 0;
+      accessToken = lb.getBlockToken();
+      nodes = lb.getLocations();
+      storageTypes = lb.getStorageTypes();
+
+      //
+      // Connect to first DataNode in the list.
+      //
+      success = createBlockOutputStream(nodes, storageTypes, 0L, false);
+
+      if (!success) {
+        LOG.info("Abandoning " + block);
+        dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
+            dfsClient.clientName);
+        block = null;
+        final DatanodeInfo badNode = nodes[errorState.getBadNodeIndex()];
+        LOG.info("Excluding datanode " + badNode);
+        excludedNodes.put(badNode, badNode);
+      }
+    } while (!success && --count >= 0);
+
+    if (!success) {
+      throw new IOException("Unable to create new block.");
+    }
+    return lb;
+  }
+
+  // connects to the first datanode in the pipeline
+  // Returns true if success, otherwise return failure.
+  //
+  private boolean createBlockOutputStream(DatanodeInfo[] nodes,
+      StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
+    if (nodes.length == 0) {
+      LOG.info("nodes are empty for write pipeline of " + block);
+      return false;
+    }
+    Status pipelineStatus = SUCCESS;
+    String firstBadLink = "";
+    boolean checkRestart = false;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("pipeline = " + Arrays.asList(nodes));
+    }
+
+    // persist blocks on namenode on next flush
+    persistBlocks.set(true);
+
+    int refetchEncryptionKey = 1;
+    while (true) {
+      boolean result = false;
+      DataOutputStream out = null;
+      try {
+        assert null == s : "Previous socket unclosed";
+        assert null == blockReplyStream : "Previous blockReplyStream unclosed";
+        s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
+        long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
+        long readTimeout = dfsClient.getDatanodeReadTimeout(nodes.length);
+
+        OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
+        InputStream unbufIn = NetUtils.getInputStream(s, readTimeout);
+        IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
+            unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
+        unbufOut = saslStreams.out;
+        unbufIn = saslStreams.in;
+        out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+            DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
+        blockReplyStream = new DataInputStream(unbufIn);
+
+        //
+        // Xmit header info to datanode
+        //
+
+        BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
+
+        // We cannot change the block length in 'block' as it counts the number
+        // of bytes ack'ed.
+        ExtendedBlock blockCopy = new ExtendedBlock(block);
+        blockCopy.setNumBytes(stat.getBlockSize());
+
+        boolean[] targetPinnings = getPinnings(nodes, true);
+        // send the request
+        new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
+            dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
+            nodes.length, block.getNumBytes(), bytesSent, newGS,
+            checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
+            (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
+
+        // receive ack for connect
+        BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+            PBHelperClient.vintPrefixed(blockReplyStream));
+        pipelineStatus = resp.getStatus();
+        firstBadLink = resp.getFirstBadLink();
+
+        // Got an restart OOB ack.
+        // If a node is already restarting, this status is not likely from
+        // the same node. If it is from a different node, it is not
+        // from the local datanode. Thus it is safe to treat this as a
+        // regular node error.
+        if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
+            !errorState.isRestartingNode()) {
+          checkRestart = true;
+          throw new IOException("A datanode is restarting.");
+        }
+		
+        String logInfo = "ack with firstBadLink as " + firstBadLink;
+        DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
+
+        assert null == blockStream : "Previous blockStream unclosed";
+        blockStream = out;
+        result =  true; // success
+        errorState.reset();
+      } catch (IOException ie) {
+        if (!errorState.isRestartingNode()) {
+          LOG.info("Exception in createBlockOutputStream", ie);
+        }
+        if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+          LOG.info("Will fetch a new encryption key and retry, "
+              + "encryption key was invalid when connecting to "
+              + nodes[0] + " : " + ie);
+          // The encryption key used is invalid.
+          refetchEncryptionKey--;
+          dfsClient.clearDataEncryptionKey();
+          // Don't close the socket/exclude this node just yet. Try again with
+          // a new encryption key.
+          continue;
+        }
+
+        // find the datanode that matches
+        if (firstBadLink.length() != 0) {
+          for (int i = 0; i < nodes.length; i++) {
+            // NB: Unconditionally using the xfer addr w/o hostname
+            if (firstBadLink.equals(nodes[i].getXferAddr())) {
+              errorState.setBadNodeIndex(i);
+              break;
+            }
+          }
+        } else {
+          assert checkRestart == false;
+          errorState.setBadNodeIndex(0);
+        }
+
+        final int i = errorState.getBadNodeIndex();
+        // Check whether there is a restart worth waiting for.
+        if (checkRestart && shouldWaitForRestart(i)) {
+          errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
+        }
+        errorState.setError(true);
+        lastException.set(ie);
+        result =  false;  // error
+      } finally {
+        if (!result) {
+          IOUtils.closeSocket(s);
+          s = null;
+          IOUtils.closeStream(out);
+          out = null;
+          IOUtils.closeStream(blockReplyStream);
+          blockReplyStream = null;
+        }
+      }
+      return result;
+    }
+  }
+
+  private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
+    if (favoredNodes == null) {
+      return null;
+    } else {
+      boolean[] pinnings = new boolean[nodes.length];
+      HashSet<String> favoredSet =
+          new HashSet<String>(Arrays.asList(favoredNodes));
+      for (int i = 0; i < nodes.length; i++) {
+        pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(nodes[i].getXferAddrWithHostname() +
+              " was chosen by name node (favored=" + pinnings[i] + ").");
+        }
+      }
+      if (shouldLog && !favoredSet.isEmpty()) {
+        // There is one or more favored nodes that were not allocated.
+        LOG.warn("These favored nodes were specified but not chosen: "
+            + favoredSet + " Specified favored nodes: "
+            + Arrays.toString(favoredNodes));
+
+      }
+      return pinnings;
+    }
+  }
+
+  protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+      throws IOException {
+    final DfsClientConf conf = dfsClient.getConf(); 
+    int retries = conf.getNumBlockWriteLocateFollowingRetry();
+    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+    while (true) {
+      long localstart = Time.monotonicNow();
+      while (true) {
+        try {
+          return dfsClient.namenode.addBlock(src, dfsClient.clientName,
+              block, excludedNodes, stat.getFileId(), favoredNodes);
+        } catch (RemoteException e) {
+          IOException ue =
+              e.unwrapRemoteException(FileNotFoundException.class,
+                  AccessControlException.class,
+                  NSQuotaExceededException.class,
+                  DSQuotaExceededException.class,
+                  QuotaByStorageTypeExceededException.class,
+                  UnresolvedPathException.class);
+          if (ue != e) {
+            throw ue; // no need to retry these exceptions
+          }
+
+
+          if (NotReplicatedYetException.class.getName().
+              equals(e.getClassName())) {
+            if (retries == 0) {
+              throw e;
+            } else {
+              --retries;
+              LOG.info("Exception while adding a block", e);
+              long elapsed = Time.monotonicNow() - localstart;
+              if (elapsed > 5000) {
+                LOG.info("Waiting for replication for "
+                    + (elapsed / 1000) + " seconds");
+              }
+              try {
+                LOG.warn("NotReplicatedYetException sleeping " + src
+                    + " retries left " + retries);
+                Thread.sleep(sleeptime);
+                sleeptime *= 2;
+              } catch (InterruptedException ie) {
+                LOG.warn("Caught exception", ie);
+              }
+            }
+          } else {
+            throw e;
+          }
+
+        }
+      }
+    }
+  }
+
+  /**
+   * This function sleeps for a certain amount of time when the writing
+   * pipeline is congested. The function calculates the time based on a
+   * decorrelated filter.
+   *
+   * @see
+   * <a href="http://www.awsarchitectureblog.com/2015/03/backoff.html">
+   *   http://www.awsarchitectureblog.com/2015/03/backoff.html</a>
+   */
+  private void backOffIfNecessary() throws InterruptedException {
+    int t = 0;
+    synchronized (congestedNodes) {
+      if (!congestedNodes.isEmpty()) {
+        StringBuilder sb = new StringBuilder("DataNode");
+        for (DatanodeInfo i : congestedNodes) {
+          sb.append(' ').append(i);
+        }
+        int range = Math.abs(lastCongestionBackoffTime * 3 -
+                                CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
+        int base = Math.min(lastCongestionBackoffTime * 3,
+                            CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
+        t = Math.min(CONGESTION_BACK_OFF_MAX_TIME_IN_MS,
+                     (int)(base + Math.random() * range));
+        lastCongestionBackoffTime = t;
+        sb.append(" are congested. Backing off for ").append(t).append(" ms");
+        LOG.info(sb.toString());
+        congestedNodes.clear();
+      }
+    }
+    if (t != 0) {
+      Thread.sleep(t);
+    }
+  }
+
+  /**
+   * get the block this streamer is writing to
+   *
+   * @return the block this streamer is writing to
+   */
+  ExtendedBlock getBlock() {
+    return block;
+  }
+
+  /**
+   * return the target datanodes in the pipeline
+   *
+   * @return the target datanodes in the pipeline
+   */
+  DatanodeInfo[] getNodes() {
+    return nodes;
+  }
+
+  /**
+   * return the token of the block
+   *
+   * @return the token of the block
+   */
+  Token<BlockTokenIdentifier> getBlockToken() {
+    return accessToken;
+  }
+
+  /**
+   * Put a packet to the data queue
+   *
+   * @param packet the packet to be put into the data queued
+   */
+  void queuePacket(DFSPacket packet) {
+    synchronized (dataQueue) {
+      if (packet == null) return;
+      packet.addTraceParent(Trace.currentSpan());
+      dataQueue.addLast(packet);
+      lastQueuedSeqno = packet.getSeqno();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Queued packet " + packet.getSeqno());
+      }
+      dataQueue.notifyAll();
+    }
+  }
+
+  /**
+   * For heartbeat packets, create buffer directly by new byte[]
+   * since heartbeats should not be blocked.
+   */
+  private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
+    final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
+    return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
+  }
+
+  private static LoadingCache<DatanodeInfo, DatanodeInfo> initExcludedNodes(
+      long excludedNodesCacheExpiry) {
+    return CacheBuilder.newBuilder()
+        .expireAfterWrite(excludedNodesCacheExpiry, TimeUnit.MILLISECONDS)
+        .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
+          @Override
+          public void onRemoval(
+              RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
+            LOG.info("Removing node " + notification.getKey()
+                + " from the excluded nodes list");
+          }
+        }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
+          @Override
+          public DatanodeInfo load(DatanodeInfo key) throws Exception {
+            return key;
+          }
+        });
+  }
+
+  private static <T> void arraycopy(T[] srcs, T[] dsts, int skipIndex) {
+    System.arraycopy(srcs, 0, dsts, 0, skipIndex);
+    System.arraycopy(srcs, skipIndex+1, dsts, skipIndex, dsts.length-skipIndex);
+  }
+
+  /**
+   * check if to persist blocks on namenode
+   *
+   * @return if to persist blocks on namenode
+   */
+  AtomicBoolean getPersistBlocks(){
+    return persistBlocks;
+  }
+
+  /**
+   * check if to append a chunk
+   *
+   * @param appendChunk if to append a chunk
+   */
+  void setAppendChunk(boolean appendChunk){
+    this.appendChunk = appendChunk;
+  }
+
+  /**
+   * get if to append a chunk
+   *
+   * @return if to append a chunk
+   */
+  boolean getAppendChunk(){
+    return appendChunk;
+  }
+
+  /**
+   * @return the last exception
+   */
+  LastExceptionInStreamer getLastException(){
+    return lastException;
+  }
+
+  /**
+   * set socket to null
+   */
+  void setSocketToNull() {
+    this.s = null;
+  }
+
+  /**
+   * return current sequence number and then increase it by 1
+   *
+   * @return current sequence number before increasing
+   */
+  long getAndIncCurrentSeqno() {
+    long old = this.currentSeqno;
+    this.currentSeqno++;
+    return old;
+  }
+
+  /**
+   * get last queued sequence number
+   *
+   * @return last queued sequence number
+   */
+  long getLastQueuedSeqno() {
+    return lastQueuedSeqno;
+  }
+
+  /**
+   * get the number of bytes of current block
+   *
+   * @return the number of bytes of current block
+   */
+  long getBytesCurBlock() {
+    return bytesCurBlock;
+  }
+
+  /**
+   * set the bytes of current block that have been written
+   *
+   * @param bytesCurBlock bytes of current block that have been written
+   */
+  void setBytesCurBlock(long bytesCurBlock) {
+    this.bytesCurBlock = bytesCurBlock;
+  }
+
+  /**
+   * increase bytes of current block by len.
+   *
+   * @param len how many bytes to increase to current block
+   */
+  void incBytesCurBlock(long len) {
+    this.bytesCurBlock += len;
+  }
+
+  /**
+   * set artificial slow down for unit test
+   *
+   * @param period artificial slow down
+   */
+  void setArtificialSlowdown(long period) {
+    this.artificialSlowdown = period;
+  }
+
+  /**
+   * if this streamer is to terminate
+   *
+   * @return if this streamer is to terminate
+   */
+  boolean streamerClosed(){
+    return streamerClosed;
+  }
+
+  void closeSocket() throws IOException {
+    if (s != null) {
+      s.close();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return  (block == null? null: block.getLocalBlock())
+        + "@" + Arrays.toString(getNodes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java
new file mode 100644
index 0000000..4a84f06
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Load default HDFS configuration resources.
+ */
+@InterfaceAudience.Private
+class HdfsConfigurationLoader {
+
+  static {
+    // adds the default resources
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+  }
+
+  /**
+   * This method is here so that when invoked, default resources are added if
+   * they haven't already been previously loaded.  Upon loading this class, the
+   * static initializer block above will be executed to add the default
+   * resources. It is safe for this method to be called multiple times
+   * as the static initializer block will only get invoked once.
+   */
+  public static void init() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
new file mode 100644
index 0000000..f03e179
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+@InterfaceAudience.Private
+public interface RemotePeerFactory {
+  /**
+   * @param addr          The address to connect to.
+   * @param blockToken    Token used during optional SASL negotiation
+   * @param datanodeId    ID of destination DataNode
+   * @return              A new Peer connected to the address.
+   *
+   * @throws IOException  If there was an error connecting or creating 
+   *                      the remote socket, encrypted stream, etc.
+   */
+  Peer newConnectedPeer(InetSocketAddress addr,
+      Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
new file mode 100644
index 0000000..ec17bb8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when an unknown cipher suite is encountered.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class UnknownCipherSuiteException extends IOException {
+  public UnknownCipherSuiteException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
new file mode 100644
index 0000000..0aac8c8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class UnknownCryptoProtocolVersionException extends IOException {
+  private static final long serialVersionUID = 8957192l;
+
+  public UnknownCryptoProtocolVersionException() {
+    super();
+  }
+
+  public UnknownCryptoProtocolVersionException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
new file mode 100644
index 0000000..2655c40
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttr.NameSpace;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+@InterfaceAudience.Private
+public class XAttrHelper {
+  
+  /**
+   * Build <code>XAttr</code> from xattr name with prefix.
+   */
+  public static XAttr buildXAttr(String name) {
+    return buildXAttr(name, null);
+  }
+  
+  /**
+   * Build <code>XAttr</code> from name with prefix and value.
+   * Name can not be null. Value can be null. The name and prefix 
+   * are validated.
+   * Both name and namespace are case sensitive.
+   */
+  public static XAttr buildXAttr(String name, byte[] value) {
+    Preconditions.checkNotNull(name, "XAttr name cannot be null.");
+    
+    final int prefixIndex = name.indexOf(".");
+    if (prefixIndex < 3) {// Prefix length is at least 3.
+      throw new HadoopIllegalArgumentException("An XAttr name must be " +
+          "prefixed with user/trusted/security/system/raw, followed by a '.'");
+    } else if (prefixIndex == name.length() - 1) {
+      throw new HadoopIllegalArgumentException("XAttr name cannot be empty.");
+    }
+    
+    NameSpace ns;
+    final String prefix = name.substring(0, prefixIndex);
+    if (StringUtils.equalsIgnoreCase(prefix, NameSpace.USER.toString())) {
+      ns = NameSpace.USER;
+    } else if (
+        StringUtils.equalsIgnoreCase(prefix, NameSpace.TRUSTED.toString())) {
+      ns = NameSpace.TRUSTED;
+    } else if (
+        StringUtils.equalsIgnoreCase(prefix, NameSpace.SYSTEM.toString())) {
+      ns = NameSpace.SYSTEM;
+    } else if (
+        StringUtils.equalsIgnoreCase(prefix, NameSpace.SECURITY.toString())) {
+      ns = NameSpace.SECURITY;
+    } else if (
+        StringUtils.equalsIgnoreCase(prefix, NameSpace.RAW.toString())) {
+      ns = NameSpace.RAW;
+    } else {
+      throw new HadoopIllegalArgumentException("An XAttr name must be " +
+          "prefixed with user/trusted/security/system/raw, followed by a '.'");
+    }
+    XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
+        substring(prefixIndex + 1)).setValue(value).build();
+    
+    return xAttr;
+  }
+  
+  /**
+   * Build xattr name with prefix as <code>XAttr</code> list.
+   */
+  public static List<XAttr> buildXAttrAsList(String name) {
+    XAttr xAttr = buildXAttr(name);
+    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+    xAttrs.add(xAttr);
+    
+    return xAttrs;
+  }
+  
+  /**
+   * Get value of first xattr from <code>XAttr</code> list
+   */
+  public static byte[] getFirstXAttrValue(List<XAttr> xAttrs) {
+    byte[] value = null;
+    XAttr xAttr = getFirstXAttr(xAttrs);
+    if (xAttr != null) {
+      value = xAttr.getValue();
+      if (value == null) {
+        value = new byte[0]; // xattr exists, but no value.
+      }
+    }
+    return value;
+  }
+  
+  /**
+   * Get first xattr from <code>XAttr</code> list
+   */
+  public static XAttr getFirstXAttr(List<XAttr> xAttrs) {
+    if (xAttrs != null && !xAttrs.isEmpty()) {
+      return xAttrs.get(0);
+    }
+    
+    return null;
+  }
+  
+  /**
+   * Build xattr map from <code>XAttr</code> list, the key is 
+   * xattr name with prefix, and value is xattr value. 
+   */
+  public static Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
+    if (xAttrs == null) {
+      return null;
+    }
+    Map<String, byte[]> xAttrMap = Maps.newHashMap();
+    for (XAttr xAttr : xAttrs) {
+      String name = getPrefixedName(xAttr);
+      byte[] value = xAttr.getValue();
+      if (value == null) {
+        value = new byte[0];
+      }
+      xAttrMap.put(name, value);
+    }
+    
+    return xAttrMap;
+  }
+  
+  /**
+   * Get name with prefix from <code>XAttr</code>
+   */
+  public static String getPrefixedName(XAttr xAttr) {
+    if (xAttr == null) {
+      return null;
+    }
+
+    return getPrefixedName(xAttr.getNameSpace(), xAttr.getName());
+  }
+
+  public static String getPrefixedName(XAttr.NameSpace ns, String name) {
+    return StringUtils.toLowerCase(ns.toString()) + "." + name;
+  }
+
+  /**
+   * Build <code>XAttr</code> list from xattr name list.
+   */
+  public static List<XAttr> buildXAttrs(List<String> names) {
+    if (names == null || names.isEmpty()) {
+      throw new HadoopIllegalArgumentException("XAttr names can not be " +
+          "null or empty.");
+    }
+    
+    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
+    for (String name : names) {
+      xAttrs.add(buildXAttr(name, null));
+    }
+    return xAttrs;
+  } 
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
new file mode 100644
index 0000000..e8ac686
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client;
+
+import java.io.InputStream;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.crypto.CryptoInputStream;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The Hdfs implementation of {@link FSDataInputStream}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HdfsDataInputStream extends FSDataInputStream {
+  public HdfsDataInputStream(DFSInputStream in) throws IOException {
+    super(in);
+  }
+
+  public HdfsDataInputStream(CryptoInputStream in) throws IOException {
+    super(in);
+    Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
+        "CryptoInputStream should wrap a DFSInputStream");
+  }
+
+  private DFSInputStream getDFSInputStream() {
+    if (in instanceof CryptoInputStream) {
+      return (DFSInputStream) ((CryptoInputStream) in).getWrappedStream();
+    }
+    return (DFSInputStream) in;
+  }
+
+  /**
+   * Get a reference to the wrapped output stream. We always want to return the
+   * actual underlying InputStream, even when we're using a CryptoStream. e.g.
+   * in the delegated methods below.
+   *
+   * @return the underlying output stream
+   */
+  public InputStream getWrappedStream() {
+      return in;
+  }
+
+  /**
+   * Get the datanode from which the stream is currently reading.
+   */
+  public DatanodeInfo getCurrentDatanode() {
+    return getDFSInputStream().getCurrentDatanode();
+  }
+
+  /**
+   * Get the block containing the target position.
+   */
+  public ExtendedBlock getCurrentBlock() {
+    return getDFSInputStream().getCurrentBlock();
+  }
+
+  /**
+   * Get the collection of blocks that has already been located.
+   */
+  public List<LocatedBlock> getAllBlocks() throws IOException {
+    return getDFSInputStream().getAllBlocks();
+  }
+
+  /**
+   * Get the visible length of the file. It will include the length of the last
+   * block even if that is in UnderConstruction state.
+   * 
+   * @return The visible length of the file.
+   */
+  public long getVisibleLength() throws IOException {
+    return getDFSInputStream().getFileLength();
+  }
+
+  /**
+   * Get statistics about the reads which this DFSInputStream has done.
+   * Note that because HdfsDataInputStream is buffered, these stats may
+   * be higher than you would expect just by adding up the number of
+   * bytes read through HdfsDataInputStream.
+   */
+  public DFSInputStream.ReadStatistics getReadStatistics() {
+    return getDFSInputStream().getReadStatistics();
+  }
+
+  public void clearReadStatistics() {
+    getDFSInputStream().clearReadStatistics();
+  }
+}


[02/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
deleted file mode 100644
index 4a016bd..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ /dev/null
@@ -1,1903 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.NullScope;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceInfo;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
-
-/*********************************************************************
- *
- * The DataStreamer class is responsible for sending data packets to the
- * datanodes in the pipeline. It retrieves a new blockid and block locations
- * from the namenode, and starts streaming packets to the pipeline of
- * Datanodes. Every packet has a sequence number associated with
- * it. When all the packets for a block are sent out and acks for each
- * if them are received, the DataStreamer closes the current block.
- *
- * The DataStreamer thread picks up packets from the dataQueue, sends it to
- * the first datanode in the pipeline and moves it from the dataQueue to the
- * ackQueue. The ResponseProcessor receives acks from the datanodes. When an
- * successful ack for a packet is received from all datanodes, the
- * ResponseProcessor removes the corresponding packet from the ackQueue.
- *
- * In case of error, all outstanding packets are moved from ackQueue. A new
- * pipeline is setup by eliminating the bad datanode from the original
- * pipeline. The DataStreamer now starts sending packets from the dataQueue.
- *
- *********************************************************************/
-
-@InterfaceAudience.Private
-class DataStreamer extends Daemon {
-  static final Log LOG = LogFactory.getLog(DataStreamer.class);
-
-  /**
-   * Create a socket for a write pipeline
-   *
-   * @param first the first datanode
-   * @param length the pipeline length
-   * @param client client
-   * @return the socket connected to the first datanode
-   */
-  static Socket createSocketForPipeline(final DatanodeInfo first,
-      final int length, final DFSClient client) throws IOException {
-    final DfsClientConf conf = client.getConf();
-    final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr);
-    }
-    final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
-    final Socket sock = client.socketFactory.createSocket();
-    final int timeout = client.getDatanodeReadTimeout(length);
-    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
-    sock.setSoTimeout(timeout);
-    sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Send buf size " + sock.getSendBufferSize());
-    }
-    return sock;
-  }
-
-  /**
-   * if this file is lazy persist
-   *
-   * @param stat the HdfsFileStatus of a file
-   * @return if this file is lazy persist
-   */
-  static boolean isLazyPersist(HdfsFileStatus stat) {
-    return stat.getStoragePolicy() == HdfsConstants.MEMORY_STORAGE_POLICY_ID;
-  }
-
-  /**
-   * release a list of packets to ByteArrayManager
-   *
-   * @param packets packets to be release
-   * @param bam ByteArrayManager
-   */
-  private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam) {
-    for(DFSPacket p : packets) {
-      p.releaseBuffer(bam);
-    }
-    packets.clear();
-  }
-  
-  static class LastExceptionInStreamer {
-    private IOException thrown;
-
-    synchronized void set(Throwable t) {
-      assert t != null;
-      this.thrown = t instanceof IOException ?
-          (IOException) t : new IOException(t);
-    }
-
-    synchronized void clear() {
-      thrown = null;
-    }
-
-    /** Check if there already is an exception. */
-    synchronized void check(boolean resetToNull) throws IOException {
-      if (thrown != null) {
-        if (LOG.isTraceEnabled()) {
-          // wrap and print the exception to know when the check is called
-          LOG.trace("Got Exception while checking", new Throwable(thrown));
-        }
-        final IOException e = thrown;
-        if (resetToNull) {
-          thrown = null;
-        }
-        throw e;
-      }
-    }
-
-    synchronized void throwException4Close() throws IOException {
-      check(false);
-      throw new ClosedChannelException();
-    }
-  }
-
-  static class ErrorState {
-    private boolean error = false;
-    private int badNodeIndex = -1;
-    private int restartingNodeIndex = -1;
-    private long restartingNodeDeadline = 0;
-    private final long datanodeRestartTimeout;
-
-    ErrorState(long datanodeRestartTimeout) {
-      this.datanodeRestartTimeout = datanodeRestartTimeout;
-    }
-
-    synchronized void reset() {
-      error = false;
-      badNodeIndex = -1;
-      restartingNodeIndex = -1;
-      restartingNodeDeadline = 0;
-    }
-
-    synchronized boolean hasError() {
-      return error;
-    }
-
-    synchronized boolean hasDatanodeError() {
-      return error && isNodeMarked();
-    }
-
-    synchronized void setError(boolean err) {
-      this.error = err;
-    }
-
-    synchronized void setBadNodeIndex(int index) {
-      this.badNodeIndex = index;
-    }
-
-    synchronized int getBadNodeIndex() {
-      return badNodeIndex;
-    }
-
-    synchronized int getRestartingNodeIndex() {
-      return restartingNodeIndex;
-    }
-
-    synchronized void initRestartingNode(int i, String message) {
-      restartingNodeIndex = i;
-      restartingNodeDeadline =  Time.monotonicNow() + datanodeRestartTimeout;
-      // If the data streamer has already set the primary node
-      // bad, clear it. It is likely that the write failed due to
-      // the DN shutdown. Even if it was a real failure, the pipeline
-      // recovery will take care of it.
-      badNodeIndex = -1;
-      LOG.info(message);
-    }
-
-    synchronized boolean isRestartingNode() {
-      return restartingNodeIndex >= 0;
-    }
-
-    synchronized boolean isNodeMarked() {
-      return badNodeIndex >= 0 || isRestartingNode();
-    }
-
-    /**
-     * This method is used when no explicit error report was received, but
-     * something failed. The first node is a suspect or unsure about the cause
-     * so that it is marked as failed.
-     */
-    synchronized void markFirstNodeIfNotMarked() {
-      // There should be no existing error and no ongoing restart.
-      if (!isNodeMarked()) {
-        badNodeIndex = 0;
-      }
-    }
-
-    synchronized void adjustState4RestartingNode() {
-      // Just took care of a node error while waiting for a node restart
-      if (restartingNodeIndex >= 0) {
-        // If the error came from a node further away than the restarting
-        // node, the restart must have been complete.
-        if (badNodeIndex > restartingNodeIndex) {
-          restartingNodeIndex = -1;
-        } else if (badNodeIndex < restartingNodeIndex) {
-          // the node index has shifted.
-          restartingNodeIndex--;
-        } else {
-          throw new IllegalStateException("badNodeIndex = " + badNodeIndex
-              + " = restartingNodeIndex = " + restartingNodeIndex);
-        }
-      }
-
-      if (!isRestartingNode()) {
-        error = false;
-      }
-      badNodeIndex = -1;
-    }
-
-    synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
-      if (restartingNodeIndex >= 0) {
-        if (!error) {
-          throw new IllegalStateException("error=false while checking" +
-              " restarting node deadline");
-        }
-
-        // check badNodeIndex
-        if (badNodeIndex == restartingNodeIndex) {
-          // ignore, if came from the restarting node
-          badNodeIndex = -1;
-        }
-        // not within the deadline
-        if (Time.monotonicNow() >= restartingNodeDeadline) {
-          // expired. declare the restarting node dead
-          restartingNodeDeadline = 0;
-          final int i = restartingNodeIndex;
-          restartingNodeIndex = -1;
-          LOG.warn("Datanode " + i + " did not restart within "
-              + datanodeRestartTimeout + "ms: " + nodes[i]);
-          // Mark the restarting node as failed. If there is any other failed
-          // node during the last pipeline construction attempt, it will not be
-          // overwritten/dropped. In this case, the restarting node will get
-          // excluded in the following attempt, if it still does not come up.
-          if (badNodeIndex == -1) {
-            badNodeIndex = i;
-          }
-        }
-      }
-    }
-  }
-
-  private volatile boolean streamerClosed = false;
-  private ExtendedBlock block; // its length is number of bytes acked
-  private Token<BlockTokenIdentifier> accessToken;
-  private DataOutputStream blockStream;
-  private DataInputStream blockReplyStream;
-  private ResponseProcessor response = null;
-  private volatile DatanodeInfo[] nodes = null; // list of targets for current block
-  private volatile StorageType[] storageTypes = null;
-  private volatile String[] storageIDs = null;
-  private final ErrorState errorState;
-
-  private BlockConstructionStage stage;  // block construction stage
-  private long bytesSent = 0; // number of bytes that've been sent
-  private final boolean isLazyPersistFile;
-
-  /** Nodes have been used in the pipeline before and have failed. */
-  private final List<DatanodeInfo> failed = new ArrayList<>();
-  /** The last ack sequence number before pipeline failure. */
-  private long lastAckedSeqnoBeforeFailure = -1;
-  private int pipelineRecoveryCount = 0;
-  /** Has the current block been hflushed? */
-  private boolean isHflushed = false;
-  /** Append on an existing block? */
-  private final boolean isAppend;
-
-  private long currentSeqno = 0;
-  private long lastQueuedSeqno = -1;
-  private long lastAckedSeqno = -1;
-  private long bytesCurBlock = 0; // bytes written in current block
-  private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
-  private Socket s;
-
-  private final DFSClient dfsClient;
-  private final String src;
-  /** Only for DataTransferProtocol.writeBlock(..) */
-  private final DataChecksum checksum4WriteBlock;
-  private final Progressable progress;
-  private final HdfsFileStatus stat;
-  // appending to existing partial block
-  private volatile boolean appendChunk = false;
-  // both dataQueue and ackQueue are protected by dataQueue lock
-  private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
-  private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
-  private final AtomicReference<CachingStrategy> cachingStrategy;
-  private final ByteArrayManager byteArrayManager;
-  //persist blocks on namenode
-  private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
-  private boolean failPacket = false;
-  private final long dfsclientSlowLogThresholdMs;
-  private long artificialSlowdown = 0;
-  // List of congested data nodes. The stream will back off if the DataNodes
-  // are congested
-  private final List<DatanodeInfo> congestedNodes = new ArrayList<>();
-  private static final int CONGESTION_BACKOFF_MEAN_TIME_IN_MS = 5000;
-  private static final int CONGESTION_BACK_OFF_MAX_TIME_IN_MS =
-      CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
-  private int lastCongestionBackoffTime;
-
-  private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
-  private final String[] favoredNodes;
-
-  private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
-                       Progressable progress, DataChecksum checksum,
-                       AtomicReference<CachingStrategy> cachingStrategy,
-                       ByteArrayManager byteArrayManage,
-                       boolean isAppend, String[] favoredNodes) {
-    this.dfsClient = dfsClient;
-    this.src = src;
-    this.progress = progress;
-    this.stat = stat;
-    this.checksum4WriteBlock = checksum;
-    this.cachingStrategy = cachingStrategy;
-    this.byteArrayManager = byteArrayManage;
-    this.isLazyPersistFile = isLazyPersist(stat);
-    this.isAppend = isAppend;
-    this.favoredNodes = favoredNodes;
-
-    final DfsClientConf conf = dfsClient.getConf();
-    this.dfsclientSlowLogThresholdMs = conf.getSlowIoWarningThresholdMs();
-    this.excludedNodes = initExcludedNodes(conf.getExcludedNodesCacheExpiry());
-    this.errorState = new ErrorState(conf.getDatanodeRestartTimeout());
-  }
-
-  /**
-   * construction with tracing info
-   */
-  DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
-               String src, Progressable progress, DataChecksum checksum,
-               AtomicReference<CachingStrategy> cachingStrategy,
-               ByteArrayManager byteArrayManage, String[] favoredNodes) {
-    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage, false, favoredNodes);
-    this.block = block;
-    stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
-  }
-
-  /**
-   * Construct a data streamer for appending to the last partial block
-   * @param lastBlock last block of the file to be appended
-   * @param stat status of the file to be appended
-   * @throws IOException if error occurs
-   */
-  DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
-               String src, Progressable progress, DataChecksum checksum,
-               AtomicReference<CachingStrategy> cachingStrategy,
-               ByteArrayManager byteArrayManage) throws IOException {
-    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage, true, null);
-    stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
-    block = lastBlock.getBlock();
-    bytesSent = block.getNumBytes();
-    accessToken = lastBlock.getBlockToken();
-  }
-
-  /**
-   * Set pipeline in construction
-   *
-   * @param lastBlock the last block of a file
-   * @throws IOException
-   */
-  void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{
-    // setup pipeline to append to the last block XXX retries??
-    setPipeline(lastBlock);
-    if (nodes.length < 1) {
-      throw new IOException("Unable to retrieve blocks locations " +
-          " for last block " + block +
-          "of file " + src);
-    }
-  }
-
-  private void setPipeline(LocatedBlock lb) {
-    setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
-  }
-
-  private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
-                           String[] storageIDs) {
-    this.nodes = nodes;
-    this.storageTypes = storageTypes;
-    this.storageIDs = storageIDs;
-  }
-
-  /**
-   * Initialize for data streaming
-   */
-  private void initDataStreaming() {
-    this.setName("DataStreamer for file " + src +
-        " block " + block);
-    response = new ResponseProcessor(nodes);
-    response.start();
-    stage = BlockConstructionStage.DATA_STREAMING;
-  }
-
-  private void endBlock() {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Closing old block " + block);
-    }
-    this.setName("DataStreamer for file " + src);
-    closeResponder();
-    closeStream();
-    setPipeline(null, null, null);
-    stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
-  }
-
-  private boolean shouldStop() {
-    return streamerClosed || errorState.hasError() || !dfsClient.clientRunning;
-  }
-
-  /*
-   * streamer thread is the only thread that opens streams to datanode,
-   * and closes them. Any error recovery is also done by this thread.
-   */
-  @Override
-  public void run() {
-    long lastPacket = Time.monotonicNow();
-    TraceScope scope = NullScope.INSTANCE;
-    while (!streamerClosed && dfsClient.clientRunning) {
-      // if the Responder encountered an error, shutdown Responder
-      if (errorState.hasError() && response != null) {
-        try {
-          response.close();
-          response.join();
-          response = null;
-        } catch (InterruptedException  e) {
-          LOG.warn("Caught exception", e);
-        }
-      }
-
-      DFSPacket one;
-      try {
-        // process datanode IO errors if any
-        boolean doSleep = processDatanodeError();
-
-        final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
-        synchronized (dataQueue) {
-          // wait for a packet to be sent.
-          long now = Time.monotonicNow();
-          while ((!shouldStop() && dataQueue.size() == 0 &&
-              (stage != BlockConstructionStage.DATA_STREAMING ||
-                  stage == BlockConstructionStage.DATA_STREAMING &&
-                      now - lastPacket < halfSocketTimeout)) || doSleep ) {
-            long timeout = halfSocketTimeout - (now-lastPacket);
-            timeout = timeout <= 0 ? 1000 : timeout;
-            timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
-                timeout : 1000;
-            try {
-              dataQueue.wait(timeout);
-            } catch (InterruptedException  e) {
-              LOG.warn("Caught exception", e);
-            }
-            doSleep = false;
-            now = Time.monotonicNow();
-          }
-          if (shouldStop()) {
-            continue;
-          }
-          // get packet to be sent.
-          if (dataQueue.isEmpty()) {
-            one = createHeartbeatPacket();
-          } else {
-            try {
-              backOffIfNecessary();
-            } catch (InterruptedException e) {
-              LOG.warn("Caught exception", e);
-            }
-            one = dataQueue.getFirst(); // regular data packet
-            long parents[] = one.getTraceParents();
-            if (parents.length > 0) {
-              scope = Trace.startSpan("dataStreamer", new TraceInfo(0, parents[0]));
-              // TODO: use setParents API once it's available from HTrace 3.2
-              // scope = Trace.startSpan("dataStreamer", Sampler.ALWAYS);
-              // scope.getSpan().setParents(parents);
-            }
-          }
-        }
-
-        // get new block from namenode.
-        if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Allocating new block");
-          }
-          setPipeline(nextBlockOutputStream());
-          initDataStreaming();
-        } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Append to block " + block);
-          }
-          setupPipelineForAppendOrRecovery();
-          if (streamerClosed) {
-            continue;
-          }
-          initDataStreaming();
-        }
-
-        long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
-        if (lastByteOffsetInBlock > stat.getBlockSize()) {
-          throw new IOException("BlockSize " + stat.getBlockSize() +
-              " is smaller than data size. " +
-              " Offset of packet in block " +
-              lastByteOffsetInBlock +
-              " Aborting file " + src);
-        }
-
-        if (one.isLastPacketInBlock()) {
-          // wait for all data packets have been successfully acked
-          synchronized (dataQueue) {
-            while (!shouldStop() && ackQueue.size() != 0) {
-              try {
-                // wait for acks to arrive from datanodes
-                dataQueue.wait(1000);
-              } catch (InterruptedException  e) {
-                LOG.warn("Caught exception", e);
-              }
-            }
-          }
-          if (shouldStop()) {
-            continue;
-          }
-          stage = BlockConstructionStage.PIPELINE_CLOSE;
-        }
-
-        // send the packet
-        Span span = null;
-        synchronized (dataQueue) {
-          // move packet from dataQueue to ackQueue
-          if (!one.isHeartbeatPacket()) {
-            span = scope.detach();
-            one.setTraceSpan(span);
-            dataQueue.removeFirst();
-            ackQueue.addLast(one);
-            dataQueue.notifyAll();
-          }
-        }
-
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("DataStreamer block " + block +
-              " sending packet " + one);
-        }
-
-        // write out data to remote datanode
-        TraceScope writeScope = Trace.startSpan("writeTo", span);
-        try {
-          one.writeTo(blockStream);
-          blockStream.flush();
-        } catch (IOException e) {
-          // HDFS-3398 treat primary DN is down since client is unable to
-          // write to primary DN. If a failed or restarting node has already
-          // been recorded by the responder, the following call will have no
-          // effect. Pipeline recovery can handle only one node error at a
-          // time. If the primary node fails again during the recovery, it
-          // will be taken out then.
-          errorState.markFirstNodeIfNotMarked();
-          throw e;
-        } finally {
-          writeScope.close();
-        }
-        lastPacket = Time.monotonicNow();
-
-        // update bytesSent
-        long tmpBytesSent = one.getLastByteOffsetBlock();
-        if (bytesSent < tmpBytesSent) {
-          bytesSent = tmpBytesSent;
-        }
-
-        if (shouldStop()) {
-          continue;
-        }
-
-        // Is this block full?
-        if (one.isLastPacketInBlock()) {
-          // wait for the close packet has been acked
-          synchronized (dataQueue) {
-            while (!shouldStop() && ackQueue.size() != 0) {
-              dataQueue.wait(1000);// wait for acks to arrive from datanodes
-            }
-          }
-          if (shouldStop()) {
-            continue;
-          }
-
-          endBlock();
-        }
-        if (progress != null) { progress.progress(); }
-
-        // This is used by unit test to trigger race conditions.
-        if (artificialSlowdown != 0 && dfsClient.clientRunning) {
-          Thread.sleep(artificialSlowdown);
-        }
-      } catch (Throwable e) {
-        // Log warning if there was a real error.
-        if (!errorState.isRestartingNode()) {
-          // Since their messages are descriptive enough, do not always
-          // log a verbose stack-trace WARN for quota exceptions.
-          if (e instanceof QuotaExceededException) {
-            LOG.debug("DataStreamer Quota Exception", e);
-          } else {
-            LOG.warn("DataStreamer Exception", e);
-          }
-        }
-        lastException.set(e);
-        assert !(e instanceof NullPointerException);
-        errorState.setError(true);
-        if (!errorState.isNodeMarked()) {
-          // Not a datanode issue
-          streamerClosed = true;
-        }
-      } finally {
-        scope.close();
-      }
-    }
-    closeInternal();
-  }
-
-  private void closeInternal() {
-    closeResponder();       // close and join
-    closeStream();
-    streamerClosed = true;
-    release();
-    synchronized (dataQueue) {
-      dataQueue.notifyAll();
-    }
-  }
-
-  /**
-   * release the DFSPackets in the two queues
-   *
-   */
-  void release() {
-    synchronized (dataQueue) {
-      releaseBuffer(dataQueue, byteArrayManager);
-      releaseBuffer(ackQueue, byteArrayManager);
-    }
-  }
-
-  /**
-   * wait for the ack of seqno
-   *
-   * @param seqno the sequence number to be acked
-   * @throws IOException
-   */
-  void waitForAckedSeqno(long seqno) throws IOException {
-    TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
-    try {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Waiting for ack for: " + seqno);
-      }
-      long begin = Time.monotonicNow();
-      try {
-        synchronized (dataQueue) {
-          while (!streamerClosed) {
-            checkClosed();
-            if (lastAckedSeqno >= seqno) {
-              break;
-            }
-            try {
-              dataQueue.wait(1000); // when we receive an ack, we notify on
-              // dataQueue
-            } catch (InterruptedException ie) {
-              throw new InterruptedIOException(
-                  "Interrupted while waiting for data to be acknowledged by pipeline");
-            }
-          }
-        }
-        checkClosed();
-      } catch (ClosedChannelException e) {
-      }
-      long duration = Time.monotonicNow() - begin;
-      if (duration > dfsclientSlowLogThresholdMs) {
-        LOG.warn("Slow waitForAckedSeqno took " + duration
-            + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
-      }
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * wait for space of dataQueue and queue the packet
-   *
-   * @param packet  the DFSPacket to be queued
-   * @throws IOException
-   */
-  void waitAndQueuePacket(DFSPacket packet) throws IOException {
-    synchronized (dataQueue) {
-      try {
-        // If queue is full, then wait till we have enough space
-        boolean firstWait = true;
-        try {
-          while (!streamerClosed && dataQueue.size() + ackQueue.size() >
-              dfsClient.getConf().getWriteMaxPackets()) {
-            if (firstWait) {
-              Span span = Trace.currentSpan();
-              if (span != null) {
-                span.addTimelineAnnotation("dataQueue.wait");
-              }
-              firstWait = false;
-            }
-            try {
-              dataQueue.wait();
-            } catch (InterruptedException e) {
-              // If we get interrupted while waiting to queue data, we still need to get rid
-              // of the current packet. This is because we have an invariant that if
-              // currentPacket gets full, it will get queued before the next writeChunk.
-              //
-              // Rather than wait around for space in the queue, we should instead try to
-              // return to the caller as soon as possible, even though we slightly overrun
-              // the MAX_PACKETS length.
-              Thread.currentThread().interrupt();
-              break;
-            }
-          }
-        } finally {
-          Span span = Trace.currentSpan();
-          if ((span != null) && (!firstWait)) {
-            span.addTimelineAnnotation("end.wait");
-          }
-        }
-        checkClosed();
-        queuePacket(packet);
-      } catch (ClosedChannelException e) {
-      }
-    }
-  }
-
-  /*
-   * close the streamer, should be called only by an external thread
-   * and only after all data to be sent has been flushed to datanode.
-   *
-   * Interrupt this data streamer if force is true
-   *
-   * @param force if this data stream is forced to be closed
-   */
-  void close(boolean force) {
-    streamerClosed = true;
-    synchronized (dataQueue) {
-      dataQueue.notifyAll();
-    }
-    if (force) {
-      this.interrupt();
-    }
-  }
-
-
-  private void checkClosed() throws IOException {
-    if (streamerClosed) {
-      lastException.throwException4Close();
-    }
-  }
-
-  private void closeResponder() {
-    if (response != null) {
-      try {
-        response.close();
-        response.join();
-      } catch (InterruptedException  e) {
-        LOG.warn("Caught exception", e);
-      } finally {
-        response = null;
-      }
-    }
-  }
-
-  private void closeStream() {
-    final MultipleIOException.Builder b = new MultipleIOException.Builder();
-
-    if (blockStream != null) {
-      try {
-        blockStream.close();
-      } catch (IOException e) {
-        b.add(e);
-      } finally {
-        blockStream = null;
-      }
-    }
-    if (blockReplyStream != null) {
-      try {
-        blockReplyStream.close();
-      } catch (IOException e) {
-        b.add(e);
-      } finally {
-        blockReplyStream = null;
-      }
-    }
-    if (null != s) {
-      try {
-        s.close();
-      } catch (IOException e) {
-        b.add(e);
-      } finally {
-        s = null;
-      }
-    }
-
-    final IOException ioe = b.build();
-    if (ioe != null) {
-      lastException.set(ioe);
-    }
-  }
-
-  /**
-   * Examine whether it is worth waiting for a node to restart.
-   * @param index the node index
-   */
-  boolean shouldWaitForRestart(int index) {
-    // Only one node in the pipeline.
-    if (nodes.length == 1) {
-      return true;
-    }
-
-    // Is it a local node?
-    InetAddress addr = null;
-    try {
-      addr = InetAddress.getByName(nodes[index].getIpAddr());
-    } catch (java.net.UnknownHostException e) {
-      // we are passing an ip address. this should not happen.
-      assert false;
-    }
-
-    if (addr != null && NetUtils.isLocalAddress(addr)) {
-      return true;
-    }
-    return false;
-  }
-
-  //
-  // Processes responses from the datanodes.  A packet is removed
-  // from the ackQueue when its response arrives.
-  //
-  private class ResponseProcessor extends Daemon {
-
-    private volatile boolean responderClosed = false;
-    private DatanodeInfo[] targets = null;
-    private boolean isLastPacketInBlock = false;
-
-    ResponseProcessor (DatanodeInfo[] targets) {
-      this.targets = targets;
-    }
-
-    @Override
-    public void run() {
-
-      setName("ResponseProcessor for block " + block);
-      PipelineAck ack = new PipelineAck();
-
-      TraceScope scope = NullScope.INSTANCE;
-      while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
-        // process responses from datanodes.
-        try {
-          // read an ack from the pipeline
-          long begin = Time.monotonicNow();
-          ack.readFields(blockReplyStream);
-          long duration = Time.monotonicNow() - begin;
-          if (duration > dfsclientSlowLogThresholdMs
-              && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
-            LOG.warn("Slow ReadProcessor read fields took " + duration
-                + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
-                + ack + ", targets: " + Arrays.asList(targets));
-          } else if (LOG.isDebugEnabled()) {
-            LOG.debug("DFSClient " + ack);
-          }
-
-          long seqno = ack.getSeqno();
-          // processes response status from datanodes.
-          ArrayList<DatanodeInfo> congestedNodesFromAck = new ArrayList<>();
-          for (int i = ack.getNumOfReplies()-1; i >=0  && dfsClient.clientRunning; i--) {
-            final Status reply = PipelineAck.getStatusFromHeader(ack
-                .getHeaderFlag(i));
-            if (PipelineAck.getECNFromHeader(ack.getHeaderFlag(i)) ==
-                PipelineAck.ECN.CONGESTED) {
-              congestedNodesFromAck.add(targets[i]);
-            }
-            // Restart will not be treated differently unless it is
-            // the local node or the only one in the pipeline.
-            if (PipelineAck.isRestartOOBStatus(reply) &&
-                shouldWaitForRestart(i)) {
-              final String message = "Datanode " + i + " is restarting: "
-                  + targets[i];
-              errorState.initRestartingNode(i, message);
-              throw new IOException(message);
-            }
-            // node error
-            if (reply != SUCCESS) {
-              errorState.setBadNodeIndex(i); // mark bad datanode
-              throw new IOException("Bad response " + reply +
-                  " for " + block + " from datanode " + targets[i]);
-            }
-          }
-
-          if (!congestedNodesFromAck.isEmpty()) {
-            synchronized (congestedNodes) {
-              congestedNodes.clear();
-              congestedNodes.addAll(congestedNodesFromAck);
-            }
-          } else {
-            synchronized (congestedNodes) {
-              congestedNodes.clear();
-              lastCongestionBackoffTime = 0;
-            }
-          }
-
-          assert seqno != PipelineAck.UNKOWN_SEQNO :
-              "Ack for unknown seqno should be a failed ack: " + ack;
-          if (seqno == DFSPacket.HEART_BEAT_SEQNO) {  // a heartbeat ack
-            continue;
-          }
-
-          // a success ack for a data packet
-          DFSPacket one;
-          synchronized (dataQueue) {
-            one = ackQueue.getFirst();
-          }
-          if (one.getSeqno() != seqno) {
-            throw new IOException("ResponseProcessor: Expecting seqno " +
-                " for block " + block +
-                one.getSeqno() + " but received " + seqno);
-          }
-          isLastPacketInBlock = one.isLastPacketInBlock();
-
-          // Fail the packet write for testing in order to force a
-          // pipeline recovery.
-          if (DFSClientFaultInjector.get().failPacket() &&
-              isLastPacketInBlock) {
-            failPacket = true;
-            throw new IOException(
-                "Failing the last packet for testing.");
-          }
-
-          // update bytesAcked
-          block.setNumBytes(one.getLastByteOffsetBlock());
-
-          synchronized (dataQueue) {
-            scope = Trace.continueSpan(one.getTraceSpan());
-            one.setTraceSpan(null);
-            lastAckedSeqno = seqno;
-            ackQueue.removeFirst();
-            dataQueue.notifyAll();
-
-            one.releaseBuffer(byteArrayManager);
-          }
-        } catch (Exception e) {
-          if (!responderClosed) {
-            lastException.set(e);
-            errorState.setError(true);
-            errorState.markFirstNodeIfNotMarked();
-            synchronized (dataQueue) {
-              dataQueue.notifyAll();
-            }
-            if (!errorState.isRestartingNode()) {
-              LOG.warn("Exception for " + block, e);
-            }
-            responderClosed = true;
-          }
-        } finally {
-            scope.close();
-        }
-      }
-    }
-
-    void close() {
-      responderClosed = true;
-      this.interrupt();
-    }
-  }
-
-  /**
-   * If this stream has encountered any errors, shutdown threads
-   * and mark the stream as closed.
-   *
-   * @return true if it should sleep for a while after returning.
-   */
-  private boolean processDatanodeError() throws IOException {
-    if (!errorState.hasDatanodeError()) {
-      return false;
-    }
-    if (response != null) {
-      LOG.info("Error Recovery for " + block +
-          " waiting for responder to exit. ");
-      return true;
-    }
-    closeStream();
-
-    // move packets from ack queue to front of the data queue
-    synchronized (dataQueue) {
-      dataQueue.addAll(0, ackQueue);
-      ackQueue.clear();
-    }
-
-    // Record the new pipeline failure recovery.
-    if (lastAckedSeqnoBeforeFailure != lastAckedSeqno) {
-      lastAckedSeqnoBeforeFailure = lastAckedSeqno;
-      pipelineRecoveryCount = 1;
-    } else {
-      // If we had to recover the pipeline five times in a row for the
-      // same packet, this client likely has corrupt data or corrupting
-      // during transmission.
-      if (++pipelineRecoveryCount > 5) {
-        LOG.warn("Error recovering pipeline for writing " +
-            block + ". Already retried 5 times for the same packet.");
-        lastException.set(new IOException("Failing write. Tried pipeline " +
-            "recovery 5 times without success."));
-        streamerClosed = true;
-        return false;
-      }
-    }
-    boolean doSleep = setupPipelineForAppendOrRecovery();
-
-    if (!streamerClosed && dfsClient.clientRunning) {
-      if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
-
-        // If we had an error while closing the pipeline, we go through a fast-path
-        // where the BlockReceiver does not run. Instead, the DataNode just finalizes
-        // the block immediately during the 'connect ack' process. So, we want to pull
-        // the end-of-block packet from the dataQueue, since we don't actually have
-        // a true pipeline to send it over.
-        //
-        // We also need to set lastAckedSeqno to the end-of-block Packet's seqno, so that
-        // a client waiting on close() will be aware that the flush finished.
-        synchronized (dataQueue) {
-          DFSPacket endOfBlockPacket = dataQueue.remove();  // remove the end of block packet
-          Span span = endOfBlockPacket.getTraceSpan();
-          if (span != null) {
-            // Close any trace span associated with this Packet
-            TraceScope scope = Trace.continueSpan(span);
-            scope.close();
-          }
-          assert endOfBlockPacket.isLastPacketInBlock();
-          assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
-          lastAckedSeqno = endOfBlockPacket.getSeqno();
-          dataQueue.notifyAll();
-        }
-        endBlock();
-      } else {
-        initDataStreaming();
-      }
-    }
-
-    return doSleep;
-  }
-
-  void setHflush() {
-    isHflushed = true;
-  }
-
-  private int findNewDatanode(final DatanodeInfo[] original
-  ) throws IOException {
-    if (nodes.length != original.length + 1) {
-      throw new IOException(
-          new StringBuilder()
-              .append("Failed to replace a bad datanode on the existing pipeline ")
-              .append("due to no more good datanodes being available to try. ")
-              .append("(Nodes: current=").append(Arrays.asList(nodes))
-              .append(", original=").append(Arrays.asList(original)).append("). ")
-              .append("The current failed datanode replacement policy is ")
-              .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
-              .append("a client may configure this via '")
-              .append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
-              .append("' in its configuration.")
-              .toString());
-    }
-    for(int i = 0; i < nodes.length; i++) {
-      int j = 0;
-      for(; j < original.length && !nodes[i].equals(original[j]); j++);
-      if (j == original.length) {
-        return i;
-      }
-    }
-    throw new IOException("Failed: new datanode not found: nodes="
-        + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
-  }
-
-  private void addDatanode2ExistingPipeline() throws IOException {
-    if (DataTransferProtocol.LOG.isDebugEnabled()) {
-      DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
-    }
-      /*
-       * Is data transfer necessary?  We have the following cases.
-       *
-       * Case 1: Failure in Pipeline Setup
-       * - Append
-       *    + Transfer the stored replica, which may be a RBW or a finalized.
-       * - Create
-       *    + If no data, then no transfer is required.
-       *    + If there are data written, transfer RBW. This case may happens
-       *      when there are streaming failure earlier in this pipeline.
-       *
-       * Case 2: Failure in Streaming
-       * - Append/Create:
-       *    + transfer RBW
-       *
-       * Case 3: Failure in Close
-       * - Append/Create:
-       *    + no transfer, let NameNode replicates the block.
-       */
-    if (!isAppend && lastAckedSeqno < 0
-        && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-      //no data have been written
-      return;
-    } else if (stage == BlockConstructionStage.PIPELINE_CLOSE
-        || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
-      //pipeline is closing
-      return;
-    }
-
-    //get a new datanode
-    final DatanodeInfo[] original = nodes;
-    final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
-        src, stat.getFileId(), block, nodes, storageIDs,
-        failed.toArray(new DatanodeInfo[failed.size()]),
-        1, dfsClient.clientName);
-    setPipeline(lb);
-
-    //find the new datanode
-    final int d = findNewDatanode(original);
-
-    //transfer replica
-    final DatanodeInfo src = d == 0? nodes[1]: nodes[d - 1];
-    final DatanodeInfo[] targets = {nodes[d]};
-    final StorageType[] targetStorageTypes = {storageTypes[d]};
-    transfer(src, targets, targetStorageTypes, lb.getBlockToken());
-  }
-
-  private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
-                        final StorageType[] targetStorageTypes,
-                        final Token<BlockTokenIdentifier> blockToken) throws IOException {
-    //transfer replica to the new datanode
-    Socket sock = null;
-    DataOutputStream out = null;
-    DataInputStream in = null;
-    try {
-      sock = createSocketForPipeline(src, 2, dfsClient);
-      final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
-      final long readTimeout = dfsClient.getDatanodeReadTimeout(2);
-
-      OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
-      InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);
-      IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
-          unbufOut, unbufIn, dfsClient, blockToken, src);
-      unbufOut = saslStreams.out;
-      unbufIn = saslStreams.in;
-      out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-          DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
-      in = new DataInputStream(unbufIn);
-
-      //send the TRANSFER_BLOCK request
-      new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
-          targets, targetStorageTypes);
-      out.flush();
-
-      //ack
-      BlockOpResponseProto response =
-          BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
-      if (SUCCESS != response.getStatus()) {
-        throw new IOException("Failed to add a datanode");
-      }
-    } finally {
-      IOUtils.closeStream(in);
-      IOUtils.closeStream(out);
-      IOUtils.closeSocket(sock);
-    }
-  }
-
-  /**
-   * Open a DataStreamer to a DataNode pipeline so that
-   * it can be written to.
-   * This happens when a file is appended or data streaming fails
-   * It keeps on trying until a pipeline is setup
-   */
-  private boolean setupPipelineForAppendOrRecovery() throws IOException {
-    // check number of datanodes
-    if (nodes == null || nodes.length == 0) {
-      String msg = "Could not get block locations. " + "Source file \""
-          + src + "\" - Aborting...";
-      LOG.warn(msg);
-      lastException.set(new IOException(msg));
-      streamerClosed = true;
-      return false;
-    }
-
-    boolean success = false;
-    long newGS = 0L;
-    while (!success && !streamerClosed && dfsClient.clientRunning) {
-      if (!handleRestartingDatanode()) {
-        return false;
-      }
-
-      final boolean isRecovery = errorState.hasError();
-      if (!handleBadDatanode()) {
-        return false;
-      }
-
-      handleDatanodeReplacement();
-
-      // get a new generation stamp and an access token
-      final LocatedBlock lb = updateBlockForPipeline();
-      newGS = lb.getBlock().getGenerationStamp();
-      accessToken = lb.getBlockToken();
-
-      // set up the pipeline again with the remaining nodes
-      success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
-
-      failPacket4Testing();
-
-      errorState.checkRestartingNodeDeadline(nodes);
-    } // while
-
-    if (success) {
-      block = updatePipeline(newGS);
-    }
-    return false; // do not sleep, continue processing
-  }
-
-  /**
-   * Sleep if a node is restarting.
-   * This process is repeated until the deadline or the node starts back up.
-   * @return true if it should continue.
-   */
-  private boolean handleRestartingDatanode() {
-    if (errorState.isRestartingNode()) {
-      // 4 seconds or the configured deadline period, whichever is shorter.
-      // This is the retry interval and recovery will be retried in this
-      // interval until timeout or success.
-      final long delay = Math.min(errorState.datanodeRestartTimeout, 4000L);
-      try {
-        Thread.sleep(delay);
-      } catch (InterruptedException ie) {
-        lastException.set(new IOException(
-            "Interrupted while waiting for restarting "
-            + nodes[errorState.getRestartingNodeIndex()]));
-        streamerClosed = true;
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Remove bad node from list of nodes if badNodeIndex was set.
-   * @return true if it should continue.
-   */
-  private boolean handleBadDatanode() {
-    final int badNodeIndex = errorState.getBadNodeIndex();
-    if (badNodeIndex >= 0) {
-      if (nodes.length <= 1) {
-        lastException.set(new IOException("All datanodes "
-            + Arrays.toString(nodes) + " are bad. Aborting..."));
-        streamerClosed = true;
-        return false;
-      }
-
-      LOG.warn("Error Recovery for " + block + " in pipeline "
-          + Arrays.toString(nodes) + ": datanode " + badNodeIndex
-          + "("+ nodes[badNodeIndex] + ") is bad.");
-      failed.add(nodes[badNodeIndex]);
-
-      DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
-      arraycopy(nodes, newnodes, badNodeIndex);
-
-      final StorageType[] newStorageTypes = new StorageType[newnodes.length];
-      arraycopy(storageTypes, newStorageTypes, badNodeIndex);
-
-      final String[] newStorageIDs = new String[newnodes.length];
-      arraycopy(storageIDs, newStorageIDs, badNodeIndex);
-
-      setPipeline(newnodes, newStorageTypes, newStorageIDs);
-
-      errorState.adjustState4RestartingNode();
-      lastException.clear();
-    }
-    return true;
-  }
-
-  /** Add a datanode if replace-datanode policy is satisfied. */
-  private void handleDatanodeReplacement() throws IOException {
-    if (dfsClient.dtpReplaceDatanodeOnFailure.satisfy(stat.getReplication(),
-        nodes, isAppend, isHflushed)) {
-      try {
-        addDatanode2ExistingPipeline();
-      } catch(IOException ioe) {
-        if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
-          throw ioe;
-        }
-        LOG.warn("Failed to replace datanode."
-            + " Continue with the remaining datanodes since "
-            + BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
-            + " is set to true.", ioe);
-      }
-    }
-  }
-
-  private void failPacket4Testing() {
-    if (failPacket) { // for testing
-      failPacket = false;
-      try {
-        // Give DNs time to send in bad reports. In real situations,
-        // good reports should follow bad ones, if client committed
-        // with those nodes.
-        Thread.sleep(2000);
-      } catch (InterruptedException ie) {}
-    }
-  }
-
-  LocatedBlock updateBlockForPipeline() throws IOException {
-    return dfsClient.namenode.updateBlockForPipeline(
-        block, dfsClient.clientName);
-  }
-
-  /** update pipeline at the namenode */
-  ExtendedBlock updatePipeline(long newGS) throws IOException {
-    final ExtendedBlock newBlock = new ExtendedBlock(
-        block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
-    dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
-        nodes, storageIDs);
-    return newBlock;
-  }
-
-  /**
-   * Open a DataStreamer to a DataNode so that it can be written to.
-   * This happens when a file is created and each time a new block is allocated.
-   * Must get block ID and the IDs of the destinations from the namenode.
-   * Returns the list of target datanodes.
-   */
-  private LocatedBlock nextBlockOutputStream() throws IOException {
-    LocatedBlock lb = null;
-    DatanodeInfo[] nodes = null;
-    StorageType[] storageTypes = null;
-    int count = dfsClient.getConf().getNumBlockWriteRetry();
-    boolean success = false;
-    ExtendedBlock oldBlock = block;
-    do {
-      errorState.reset();
-      lastException.clear();
-      success = false;
-
-      DatanodeInfo[] excluded =
-          excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
-              .keySet()
-              .toArray(new DatanodeInfo[0]);
-      block = oldBlock;
-      lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
-      block = lb.getBlock();
-      block.setNumBytes(0);
-      bytesSent = 0;
-      accessToken = lb.getBlockToken();
-      nodes = lb.getLocations();
-      storageTypes = lb.getStorageTypes();
-
-      //
-      // Connect to first DataNode in the list.
-      //
-      success = createBlockOutputStream(nodes, storageTypes, 0L, false);
-
-      if (!success) {
-        LOG.info("Abandoning " + block);
-        dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
-            dfsClient.clientName);
-        block = null;
-        final DatanodeInfo badNode = nodes[errorState.getBadNodeIndex()];
-        LOG.info("Excluding datanode " + badNode);
-        excludedNodes.put(badNode, badNode);
-      }
-    } while (!success && --count >= 0);
-
-    if (!success) {
-      throw new IOException("Unable to create new block.");
-    }
-    return lb;
-  }
-
-  // connects to the first datanode in the pipeline
-  // Returns true if success, otherwise return failure.
-  //
-  private boolean createBlockOutputStream(DatanodeInfo[] nodes,
-      StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
-    if (nodes.length == 0) {
-      LOG.info("nodes are empty for write pipeline of " + block);
-      return false;
-    }
-    Status pipelineStatus = SUCCESS;
-    String firstBadLink = "";
-    boolean checkRestart = false;
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("pipeline = " + Arrays.asList(nodes));
-    }
-
-    // persist blocks on namenode on next flush
-    persistBlocks.set(true);
-
-    int refetchEncryptionKey = 1;
-    while (true) {
-      boolean result = false;
-      DataOutputStream out = null;
-      try {
-        assert null == s : "Previous socket unclosed";
-        assert null == blockReplyStream : "Previous blockReplyStream unclosed";
-        s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
-        long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
-        long readTimeout = dfsClient.getDatanodeReadTimeout(nodes.length);
-
-        OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
-        InputStream unbufIn = NetUtils.getInputStream(s, readTimeout);
-        IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
-            unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
-        unbufOut = saslStreams.out;
-        unbufIn = saslStreams.in;
-        out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
-        blockReplyStream = new DataInputStream(unbufIn);
-
-        //
-        // Xmit header info to datanode
-        //
-
-        BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
-
-        // We cannot change the block length in 'block' as it counts the number
-        // of bytes ack'ed.
-        ExtendedBlock blockCopy = new ExtendedBlock(block);
-        blockCopy.setNumBytes(stat.getBlockSize());
-
-        boolean[] targetPinnings = getPinnings(nodes, true);
-        // send the request
-        new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
-            dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
-            nodes.length, block.getNumBytes(), bytesSent, newGS,
-            checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
-            (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
-
-        // receive ack for connect
-        BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
-            PBHelperClient.vintPrefixed(blockReplyStream));
-        pipelineStatus = resp.getStatus();
-        firstBadLink = resp.getFirstBadLink();
-
-        // Got an restart OOB ack.
-        // If a node is already restarting, this status is not likely from
-        // the same node. If it is from a different node, it is not
-        // from the local datanode. Thus it is safe to treat this as a
-        // regular node error.
-        if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
-            !errorState.isRestartingNode()) {
-          checkRestart = true;
-          throw new IOException("A datanode is restarting.");
-        }
-		
-        String logInfo = "ack with firstBadLink as " + firstBadLink;
-        DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
-
-        assert null == blockStream : "Previous blockStream unclosed";
-        blockStream = out;
-        result =  true; // success
-        errorState.reset();
-      } catch (IOException ie) {
-        if (!errorState.isRestartingNode()) {
-          LOG.info("Exception in createBlockOutputStream", ie);
-        }
-        if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
-          LOG.info("Will fetch a new encryption key and retry, "
-              + "encryption key was invalid when connecting to "
-              + nodes[0] + " : " + ie);
-          // The encryption key used is invalid.
-          refetchEncryptionKey--;
-          dfsClient.clearDataEncryptionKey();
-          // Don't close the socket/exclude this node just yet. Try again with
-          // a new encryption key.
-          continue;
-        }
-
-        // find the datanode that matches
-        if (firstBadLink.length() != 0) {
-          for (int i = 0; i < nodes.length; i++) {
-            // NB: Unconditionally using the xfer addr w/o hostname
-            if (firstBadLink.equals(nodes[i].getXferAddr())) {
-              errorState.setBadNodeIndex(i);
-              break;
-            }
-          }
-        } else {
-          assert checkRestart == false;
-          errorState.setBadNodeIndex(0);
-        }
-
-        final int i = errorState.getBadNodeIndex();
-        // Check whether there is a restart worth waiting for.
-        if (checkRestart && shouldWaitForRestart(i)) {
-          errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
-        }
-        errorState.setError(true);
-        lastException.set(ie);
-        result =  false;  // error
-      } finally {
-        if (!result) {
-          IOUtils.closeSocket(s);
-          s = null;
-          IOUtils.closeStream(out);
-          out = null;
-          IOUtils.closeStream(blockReplyStream);
-          blockReplyStream = null;
-        }
-      }
-      return result;
-    }
-  }
-
-  private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
-    if (favoredNodes == null) {
-      return null;
-    } else {
-      boolean[] pinnings = new boolean[nodes.length];
-      HashSet<String> favoredSet =
-          new HashSet<String>(Arrays.asList(favoredNodes));
-      for (int i = 0; i < nodes.length; i++) {
-        pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(nodes[i].getXferAddrWithHostname() +
-              " was chosen by name node (favored=" + pinnings[i] + ").");
-        }
-      }
-      if (shouldLog && !favoredSet.isEmpty()) {
-        // There is one or more favored nodes that were not allocated.
-        LOG.warn("These favored nodes were specified but not chosen: "
-            + favoredSet + " Specified favored nodes: "
-            + Arrays.toString(favoredNodes));
-
-      }
-      return pinnings;
-    }
-  }
-
-  protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
-      throws IOException {
-    final DfsClientConf conf = dfsClient.getConf(); 
-    int retries = conf.getNumBlockWriteLocateFollowingRetry();
-    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
-    while (true) {
-      long localstart = Time.monotonicNow();
-      while (true) {
-        try {
-          return dfsClient.namenode.addBlock(src, dfsClient.clientName,
-              block, excludedNodes, stat.getFileId(), favoredNodes);
-        } catch (RemoteException e) {
-          IOException ue =
-              e.unwrapRemoteException(FileNotFoundException.class,
-                  AccessControlException.class,
-                  NSQuotaExceededException.class,
-                  DSQuotaExceededException.class,
-                  QuotaByStorageTypeExceededException.class,
-                  UnresolvedPathException.class);
-          if (ue != e) {
-            throw ue; // no need to retry these exceptions
-          }
-
-
-          if (NotReplicatedYetException.class.getName().
-              equals(e.getClassName())) {
-            if (retries == 0) {
-              throw e;
-            } else {
-              --retries;
-              LOG.info("Exception while adding a block", e);
-              long elapsed = Time.monotonicNow() - localstart;
-              if (elapsed > 5000) {
-                LOG.info("Waiting for replication for "
-                    + (elapsed / 1000) + " seconds");
-              }
-              try {
-                LOG.warn("NotReplicatedYetException sleeping " + src
-                    + " retries left " + retries);
-                Thread.sleep(sleeptime);
-                sleeptime *= 2;
-              } catch (InterruptedException ie) {
-                LOG.warn("Caught exception", ie);
-              }
-            }
-          } else {
-            throw e;
-          }
-
-        }
-      }
-    }
-  }
-
-  /**
-   * This function sleeps for a certain amount of time when the writing
-   * pipeline is congested. The function calculates the time based on a
-   * decorrelated filter.
-   *
-   * @see
-   * <a href="http://www.awsarchitectureblog.com/2015/03/backoff.html">
-   *   http://www.awsarchitectureblog.com/2015/03/backoff.html</a>
-   */
-  private void backOffIfNecessary() throws InterruptedException {
-    int t = 0;
-    synchronized (congestedNodes) {
-      if (!congestedNodes.isEmpty()) {
-        StringBuilder sb = new StringBuilder("DataNode");
-        for (DatanodeInfo i : congestedNodes) {
-          sb.append(' ').append(i);
-        }
-        int range = Math.abs(lastCongestionBackoffTime * 3 -
-                                CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
-        int base = Math.min(lastCongestionBackoffTime * 3,
-                            CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
-        t = Math.min(CONGESTION_BACK_OFF_MAX_TIME_IN_MS,
-                     (int)(base + Math.random() * range));
-        lastCongestionBackoffTime = t;
-        sb.append(" are congested. Backing off for ").append(t).append(" ms");
-        LOG.info(sb.toString());
-        congestedNodes.clear();
-      }
-    }
-    if (t != 0) {
-      Thread.sleep(t);
-    }
-  }
-
-  /**
-   * get the block this streamer is writing to
-   *
-   * @return the block this streamer is writing to
-   */
-  ExtendedBlock getBlock() {
-    return block;
-  }
-
-  /**
-   * return the target datanodes in the pipeline
-   *
-   * @return the target datanodes in the pipeline
-   */
-  DatanodeInfo[] getNodes() {
-    return nodes;
-  }
-
-  /**
-   * return the token of the block
-   *
-   * @return the token of the block
-   */
-  Token<BlockTokenIdentifier> getBlockToken() {
-    return accessToken;
-  }
-
-  /**
-   * Put a packet to the data queue
-   *
-   * @param packet the packet to be put into the data queued
-   */
-  void queuePacket(DFSPacket packet) {
-    synchronized (dataQueue) {
-      if (packet == null) return;
-      packet.addTraceParent(Trace.currentSpan());
-      dataQueue.addLast(packet);
-      lastQueuedSeqno = packet.getSeqno();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Queued packet " + packet.getSeqno());
-      }
-      dataQueue.notifyAll();
-    }
-  }
-
-  /**
-   * For heartbeat packets, create buffer directly by new byte[]
-   * since heartbeats should not be blocked.
-   */
-  private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
-    final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
-    return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
-  }
-
-  private static LoadingCache<DatanodeInfo, DatanodeInfo> initExcludedNodes(
-      long excludedNodesCacheExpiry) {
-    return CacheBuilder.newBuilder()
-        .expireAfterWrite(excludedNodesCacheExpiry, TimeUnit.MILLISECONDS)
-        .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
-          @Override
-          public void onRemoval(
-              RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
-            LOG.info("Removing node " + notification.getKey()
-                + " from the excluded nodes list");
-          }
-        }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
-          @Override
-          public DatanodeInfo load(DatanodeInfo key) throws Exception {
-            return key;
-          }
-        });
-  }
-
-  private static <T> void arraycopy(T[] srcs, T[] dsts, int skipIndex) {
-    System.arraycopy(srcs, 0, dsts, 0, skipIndex);
-    System.arraycopy(srcs, skipIndex+1, dsts, skipIndex, dsts.length-skipIndex);
-  }
-
-  /**
-   * check if to persist blocks on namenode
-   *
-   * @return if to persist blocks on namenode
-   */
-  AtomicBoolean getPersistBlocks(){
-    return persistBlocks;
-  }
-
-  /**
-   * check if to append a chunk
-   *
-   * @param appendChunk if to append a chunk
-   */
-  void setAppendChunk(boolean appendChunk){
-    this.appendChunk = appendChunk;
-  }
-
-  /**
-   * get if to append a chunk
-   *
-   * @return if to append a chunk
-   */
-  boolean getAppendChunk(){
-    return appendChunk;
-  }
-
-  /**
-   * @return the last exception
-   */
-  LastExceptionInStreamer getLastException(){
-    return lastException;
-  }
-
-  /**
-   * set socket to null
-   */
-  void setSocketToNull() {
-    this.s = null;
-  }
-
-  /**
-   * return current sequence number and then increase it by 1
-   *
-   * @return current sequence number before increasing
-   */
-  long getAndIncCurrentSeqno() {
-    long old = this.currentSeqno;
-    this.currentSeqno++;
-    return old;
-  }
-
-  /**
-   * get last queued sequence number
-   *
-   * @return last queued sequence number
-   */
-  long getLastQueuedSeqno() {
-    return lastQueuedSeqno;
-  }
-
-  /**
-   * get the number of bytes of current block
-   *
-   * @return the number of bytes of current block
-   */
-  long getBytesCurBlock() {
-    return bytesCurBlock;
-  }
-
-  /**
-   * set the bytes of current block that have been written
-   *
-   * @param bytesCurBlock bytes of current block that have been written
-   */
-  void setBytesCurBlock(long bytesCurBlock) {
-    this.bytesCurBlock = bytesCurBlock;
-  }
-
-  /**
-   * increase bytes of current block by len.
-   *
-   * @param len how many bytes to increase to current block
-   */
-  void incBytesCurBlock(long len) {
-    this.bytesCurBlock += len;
-  }
-
-  /**
-   * set artificial slow down for unit test
-   *
-   * @param period artificial slow down
-   */
-  void setArtificialSlowdown(long period) {
-    this.artificialSlowdown = period;
-  }
-
-  /**
-   * if this streamer is to terminate
-   *
-   * @return if this streamer is to terminate
-   */
-  boolean streamerClosed(){
-    return streamerClosed;
-  }
-
-  void closeSocket() throws IOException {
-    if (s != null) {
-      s.close();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return  (block == null? null: block.getLocalBlock())
-        + "@" + Arrays.toString(getNodes());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
index ef9f27a..b6bf6cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
@@ -31,9 +31,7 @@ public class HdfsConfiguration extends Configuration {
     addDeprecatedKeys();
 
     // adds the default resources
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-
+    HdfsConfigurationLoader.init();
   }
 
   public HdfsConfiguration() {
@@ -52,9 +50,10 @@ public class HdfsConfiguration extends Configuration {
    * This method is here so that when invoked, HdfsConfiguration is class-loaded if
    * it hasn't already been previously loaded.  Upon loading the class, the static 
    * initializer block above will be executed to add the deprecated keys and to add
-   * the default resources.   It is safe for this method to be called multiple times 
-   * as the static initializer block will only get invoked once.
-   * 
+   * the default resources via {@link HdfsConfigurationLoader#init()}. It is
+   * safe for this method to be called multiple times as the static initializer
+   * block will only get invoked once.
+   *
    * This replaces the previously, dangerous practice of other classes calling
    * Configuration.addDefaultResource("hdfs-default.xml") directly without loading 
    * HdfsConfiguration class first, thereby skipping the key deprecation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
deleted file mode 100644
index f03e179..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.security.token.Token;
-
-@InterfaceAudience.Private
-public interface RemotePeerFactory {
-  /**
-   * @param addr          The address to connect to.
-   * @param blockToken    Token used during optional SASL negotiation
-   * @param datanodeId    ID of destination DataNode
-   * @return              A new Peer connected to the address.
-   *
-   * @throws IOException  If there was an error connecting or creating 
-   *                      the remote socket, encrypted stream, etc.
-   */
-  Peer newConnectedPeer(InetSocketAddress addr,
-      Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
-      throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
deleted file mode 100644
index ec17bb8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Thrown when an unknown cipher suite is encountered.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class UnknownCipherSuiteException extends IOException {
-  public UnknownCipherSuiteException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
deleted file mode 100644
index 0aac8c8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class UnknownCryptoProtocolVersionException extends IOException {
-  private static final long serialVersionUID = 8957192l;
-
-  public UnknownCryptoProtocolVersionException() {
-    super();
-  }
-
-  public UnknownCryptoProtocolVersionException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
deleted file mode 100644
index 2655c40..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttr.NameSpace;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-@InterfaceAudience.Private
-public class XAttrHelper {
-  
-  /**
-   * Build <code>XAttr</code> from xattr name with prefix.
-   */
-  public static XAttr buildXAttr(String name) {
-    return buildXAttr(name, null);
-  }
-  
-  /**
-   * Build <code>XAttr</code> from name with prefix and value.
-   * Name can not be null. Value can be null. The name and prefix 
-   * are validated.
-   * Both name and namespace are case sensitive.
-   */
-  public static XAttr buildXAttr(String name, byte[] value) {
-    Preconditions.checkNotNull(name, "XAttr name cannot be null.");
-    
-    final int prefixIndex = name.indexOf(".");
-    if (prefixIndex < 3) {// Prefix length is at least 3.
-      throw new HadoopIllegalArgumentException("An XAttr name must be " +
-          "prefixed with user/trusted/security/system/raw, followed by a '.'");
-    } else if (prefixIndex == name.length() - 1) {
-      throw new HadoopIllegalArgumentException("XAttr name cannot be empty.");
-    }
-    
-    NameSpace ns;
-    final String prefix = name.substring(0, prefixIndex);
-    if (StringUtils.equalsIgnoreCase(prefix, NameSpace.USER.toString())) {
-      ns = NameSpace.USER;
-    } else if (
-        StringUtils.equalsIgnoreCase(prefix, NameSpace.TRUSTED.toString())) {
-      ns = NameSpace.TRUSTED;
-    } else if (
-        StringUtils.equalsIgnoreCase(prefix, NameSpace.SYSTEM.toString())) {
-      ns = NameSpace.SYSTEM;
-    } else if (
-        StringUtils.equalsIgnoreCase(prefix, NameSpace.SECURITY.toString())) {
-      ns = NameSpace.SECURITY;
-    } else if (
-        StringUtils.equalsIgnoreCase(prefix, NameSpace.RAW.toString())) {
-      ns = NameSpace.RAW;
-    } else {
-      throw new HadoopIllegalArgumentException("An XAttr name must be " +
-          "prefixed with user/trusted/security/system/raw, followed by a '.'");
-    }
-    XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
-        substring(prefixIndex + 1)).setValue(value).build();
-    
-    return xAttr;
-  }
-  
-  /**
-   * Build xattr name with prefix as <code>XAttr</code> list.
-   */
-  public static List<XAttr> buildXAttrAsList(String name) {
-    XAttr xAttr = buildXAttr(name);
-    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
-    xAttrs.add(xAttr);
-    
-    return xAttrs;
-  }
-  
-  /**
-   * Get value of first xattr from <code>XAttr</code> list
-   */
-  public static byte[] getFirstXAttrValue(List<XAttr> xAttrs) {
-    byte[] value = null;
-    XAttr xAttr = getFirstXAttr(xAttrs);
-    if (xAttr != null) {
-      value = xAttr.getValue();
-      if (value == null) {
-        value = new byte[0]; // xattr exists, but no value.
-      }
-    }
-    return value;
-  }
-  
-  /**
-   * Get first xattr from <code>XAttr</code> list
-   */
-  public static XAttr getFirstXAttr(List<XAttr> xAttrs) {
-    if (xAttrs != null && !xAttrs.isEmpty()) {
-      return xAttrs.get(0);
-    }
-    
-    return null;
-  }
-  
-  /**
-   * Build xattr map from <code>XAttr</code> list, the key is 
-   * xattr name with prefix, and value is xattr value. 
-   */
-  public static Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
-    if (xAttrs == null) {
-      return null;
-    }
-    Map<String, byte[]> xAttrMap = Maps.newHashMap();
-    for (XAttr xAttr : xAttrs) {
-      String name = getPrefixedName(xAttr);
-      byte[] value = xAttr.getValue();
-      if (value == null) {
-        value = new byte[0];
-      }
-      xAttrMap.put(name, value);
-    }
-    
-    return xAttrMap;
-  }
-  
-  /**
-   * Get name with prefix from <code>XAttr</code>
-   */
-  public static String getPrefixedName(XAttr xAttr) {
-    if (xAttr == null) {
-      return null;
-    }
-
-    return getPrefixedName(xAttr.getNameSpace(), xAttr.getName());
-  }
-
-  public static String getPrefixedName(XAttr.NameSpace ns, String name) {
-    return StringUtils.toLowerCase(ns.toString()) + "." + name;
-  }
-
-  /**
-   * Build <code>XAttr</code> list from xattr name list.
-   */
-  public static List<XAttr> buildXAttrs(List<String> names) {
-    if (names == null || names.isEmpty()) {
-      throw new HadoopIllegalArgumentException("XAttr names can not be " +
-          "null or empty.");
-    }
-    
-    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
-    for (String name : names) {
-      xAttrs.add(buildXAttr(name, null));
-    }
-    return xAttrs;
-  } 
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
deleted file mode 100644
index e8ac686..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client;
-
-import java.io.InputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.crypto.CryptoInputStream;
-import org.apache.hadoop.hdfs.DFSInputStream;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-
-import com.google.common.base.Preconditions;
-
-/**
- * The Hdfs implementation of {@link FSDataInputStream}.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class HdfsDataInputStream extends FSDataInputStream {
-  public HdfsDataInputStream(DFSInputStream in) throws IOException {
-    super(in);
-  }
-
-  public HdfsDataInputStream(CryptoInputStream in) throws IOException {
-    super(in);
-    Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
-        "CryptoInputStream should wrap a DFSInputStream");
-  }
-
-  private DFSInputStream getDFSInputStream() {
-    if (in instanceof CryptoInputStream) {
-      return (DFSInputStream) ((CryptoInputStream) in).getWrappedStream();
-    }
-    return (DFSInputStream) in;
-  }
-
-  /**
-   * Get a reference to the wrapped output stream. We always want to return the
-   * actual underlying InputStream, even when we're using a CryptoStream. e.g.
-   * in the delegated methods below.
-   *
-   * @return the underlying output stream
-   */
-  public InputStream getWrappedStream() {
-      return in;
-  }
-
-  /**
-   * Get the datanode from which the stream is currently reading.
-   */
-  public DatanodeInfo getCurrentDatanode() {
-    return getDFSInputStream().getCurrentDatanode();
-  }
-
-  /**
-   * Get the block containing the target position.
-   */
-  public ExtendedBlock getCurrentBlock() {
-    return getDFSInputStream().getCurrentBlock();
-  }
-
-  /**
-   * Get the collection of blocks that has already been located.
-   */
-  public List<LocatedBlock> getAllBlocks() throws IOException {
-    return getDFSInputStream().getAllBlocks();
-  }
-
-  /**
-   * Get the visible length of the file. It will include the length of the last
-   * block even if that is in UnderConstruction state.
-   * 
-   * @return The visible length of the file.
-   */
-  public long getVisibleLength() throws IOException {
-    return getDFSInputStream().getFileLength();
-  }
-
-  /**
-   * Get statistics about the reads which this DFSInputStream has done.
-   * Note that because HdfsDataInputStream is buffered, these stats may
-   * be higher than you would expect just by adding up the number of
-   * bytes read through HdfsDataInputStream.
-   */
-  public DFSInputStream.ReadStatistics getReadStatistics() {
-    return getDFSInputStream().getReadStatistics();
-  }
-
-  public void clearReadStatistics() {
-    getDFSInputStream().clearReadStatistics();
-  }
-}


[03/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
deleted file mode 100755
index 5466936..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ /dev/null
@@ -1,923 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.channels.ClosedChannelException;
-import java.util.EnumSet;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import org.apache.hadoop.fs.CanSetDropBehind;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSOutputSummer;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Syncable;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.DataChecksum.Type;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-
-/****************************************************************
- * DFSOutputStream creates files from a stream of bytes.
- *
- * The client application writes data that is cached internally by
- * this stream. Data is broken up into packets, each packet is
- * typically 64K in size. A packet comprises of chunks. Each chunk
- * is typically 512 bytes and has an associated checksum with it.
- *
- * When a client application fills up the currentPacket, it is
- * enqueued into the dataQueue of DataStreamer. DataStreamer is a
- * thread that picks up packets from the dataQueue and sends it to
- * the first datanode in the pipeline.
- *
- ****************************************************************/
-@InterfaceAudience.Private
-public class DFSOutputStream extends FSOutputSummer
-    implements Syncable, CanSetDropBehind {
-  static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
-  /**
-   * Number of times to retry creating a file when there are transient 
-   * errors (typically related to encryption zones and KeyProvider operations).
-   */
-  @VisibleForTesting
-  static final int CREATE_RETRY_COUNT = 10;
-  @VisibleForTesting
-  static CryptoProtocolVersion[] SUPPORTED_CRYPTO_VERSIONS =
-      CryptoProtocolVersion.supported();
-
-  protected final DFSClient dfsClient;
-  protected final ByteArrayManager byteArrayManager;
-  // closed is accessed by different threads under different locks.
-  protected volatile boolean closed = false;
-
-  protected final String src;
-  protected final long fileId;
-  protected final long blockSize;
-  protected final int bytesPerChecksum;
-
-  protected DFSPacket currentPacket = null;
-  private DataStreamer streamer;
-  protected int packetSize = 0; // write packet size, not including the header.
-  protected int chunksPerPacket = 0;
-  protected long lastFlushOffset = 0; // offset when flush was invoked
-  private long initialFileSize = 0; // at time of file open
-  private final short blockReplication; // replication factor of file
-  protected boolean shouldSyncBlock = false; // force blocks to disk upon close
-  protected final AtomicReference<CachingStrategy> cachingStrategy;
-  private FileEncryptionInfo fileEncryptionInfo;
-
-  /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
-  protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
-      long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
-    final byte[] buf;
-    final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
-
-    try {
-      buf = byteArrayManager.newByteArray(bufferSize);
-    } catch (InterruptedException ie) {
-      final InterruptedIOException iioe = new InterruptedIOException(
-          "seqno=" + seqno);
-      iioe.initCause(ie);
-      throw iioe;
-    }
-
-    return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno,
-                         getChecksumSize(), lastPacketInBlock);
-  }
-
-  @Override
-  protected void checkClosed() throws IOException {
-    if (isClosed()) {
-      getStreamer().getLastException().throwException4Close();
-    }
-  }
-
-  //
-  // returns the list of targets, if any, that is being currently used.
-  //
-  @VisibleForTesting
-  public synchronized DatanodeInfo[] getPipeline() {
-    if (getStreamer().streamerClosed()) {
-      return null;
-    }
-    DatanodeInfo[] currentNodes = getStreamer().getNodes();
-    if (currentNodes == null) {
-      return null;
-    }
-    DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
-    for (int i = 0; i < currentNodes.length; i++) {
-      value[i] = currentNodes[i];
-    }
-    return value;
-  }
-
-  /** 
-   * @return the object for computing checksum.
-   *         The type is NULL if checksum is not computed.
-   */
-  private static DataChecksum getChecksum4Compute(DataChecksum checksum,
-      HdfsFileStatus stat) {
-    if (DataStreamer.isLazyPersist(stat) && stat.getReplication() == 1) {
-      // do not compute checksum for writing to single replica to memory
-      return DataChecksum.newDataChecksum(Type.NULL,
-          checksum.getBytesPerChecksum());
-    }
-    return checksum;
-  }
- 
-  private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
-      HdfsFileStatus stat, DataChecksum checksum) throws IOException {
-    super(getChecksum4Compute(checksum, stat));
-    this.dfsClient = dfsClient;
-    this.src = src;
-    this.fileId = stat.getFileId();
-    this.blockSize = stat.getBlockSize();
-    this.blockReplication = stat.getReplication();
-    this.fileEncryptionInfo = stat.getFileEncryptionInfo();
-    this.cachingStrategy = new AtomicReference<CachingStrategy>(
-        dfsClient.getDefaultWriteCachingStrategy());
-    if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug(
-          "Set non-null progress callback on DFSOutputStream " + src);
-    }
-    
-    this.bytesPerChecksum = checksum.getBytesPerChecksum();
-    if (bytesPerChecksum <= 0) {
-      throw new HadoopIllegalArgumentException(
-          "Invalid value: bytesPerChecksum = " + bytesPerChecksum + " <= 0");
-    }
-    if (blockSize % bytesPerChecksum != 0) {
-      throw new HadoopIllegalArgumentException("Invalid values: "
-          + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
-          + ") must divide block size (=" + blockSize + ").");
-    }
-    this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
-  }
-
-  /** Construct a new output stream for creating a file. */
-  protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
-      EnumSet<CreateFlag> flag, Progressable progress,
-      DataChecksum checksum, String[] favoredNodes) throws IOException {
-    this(dfsClient, src, progress, stat, checksum);
-    this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
-
-    computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
-
-    streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
-        cachingStrategy, byteArrayManager, favoredNodes);
-  }
-
-  static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
-      FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize, Progressable progress, int buffersize,
-      DataChecksum checksum, String[] favoredNodes) throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("newStreamForCreate", src);
-    try {
-      HdfsFileStatus stat = null;
-
-      // Retry the create if we get a RetryStartFileException up to a maximum
-      // number of times
-      boolean shouldRetry = true;
-      int retryCount = CREATE_RETRY_COUNT;
-      while (shouldRetry) {
-        shouldRetry = false;
-        try {
-          stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
-              new EnumSetWritable<CreateFlag>(flag), createParent, replication,
-              blockSize, SUPPORTED_CRYPTO_VERSIONS);
-          break;
-        } catch (RemoteException re) {
-          IOException e = re.unwrapRemoteException(
-              AccessControlException.class,
-              DSQuotaExceededException.class,
-              QuotaByStorageTypeExceededException.class,
-              FileAlreadyExistsException.class,
-              FileNotFoundException.class,
-              ParentNotDirectoryException.class,
-              NSQuotaExceededException.class,
-              RetryStartFileException.class,
-              SafeModeException.class,
-              UnresolvedPathException.class,
-              SnapshotAccessControlException.class,
-              UnknownCryptoProtocolVersionException.class);
-          if (e instanceof RetryStartFileException) {
-            if (retryCount > 0) {
-              shouldRetry = true;
-              retryCount--;
-            } else {
-              throw new IOException("Too many retries because of encryption" +
-                  " zone operations", e);
-            }
-          } else {
-            throw e;
-          }
-        }
-      }
-      Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
-      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
-          flag, progress, checksum, favoredNodes);
-      out.start();
-      return out;
-    } finally {
-      scope.close();
-    }
-  }
-
-  /** Construct a new output stream for append. */
-  private DFSOutputStream(DFSClient dfsClient, String src,
-      EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
-      HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
-          throws IOException {
-    this(dfsClient, src, progress, stat, checksum);
-    initialFileSize = stat.getLen(); // length of file when opened
-    this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
-
-    boolean toNewBlock = flags.contains(CreateFlag.NEW_BLOCK);
-
-    this.fileEncryptionInfo = stat.getFileEncryptionInfo();
-
-    // The last partial block of the file has to be filled.
-    if (!toNewBlock && lastBlock != null) {
-      // indicate that we are appending to an existing block
-      streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
-          cachingStrategy, byteArrayManager);
-      getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
-      adjustPacketChunkSize(stat);
-      getStreamer().setPipelineInConstruction(lastBlock);
-    } else {
-      computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
-          bytesPerChecksum);
-      streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
-          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
-          favoredNodes);
-    }
-  }
-
-  private void adjustPacketChunkSize(HdfsFileStatus stat) throws IOException{
-
-    long usedInLastBlock = stat.getLen() % blockSize;
-    int freeInLastBlock = (int)(blockSize - usedInLastBlock);
-
-    // calculate the amount of free space in the pre-existing
-    // last crc chunk
-    int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
-    int freeInCksum = bytesPerChecksum - usedInCksum;
-
-    // if there is space in the last block, then we have to
-    // append to that block
-    if (freeInLastBlock == blockSize) {
-      throw new IOException("The last block for file " +
-          src + " is full.");
-    }
-
-    if (usedInCksum > 0 && freeInCksum > 0) {
-      // if there is space in the last partial chunk, then
-      // setup in such a way that the next packet will have only
-      // one chunk that fills up the partial chunk.
-      //
-      computePacketChunkSize(0, freeInCksum);
-      setChecksumBufSize(freeInCksum);
-      getStreamer().setAppendChunk(true);
-    } else {
-      // if the remaining space in the block is smaller than
-      // that expected size of of a packet, then create
-      // smaller size packet.
-      //
-      computePacketChunkSize(
-          Math.min(dfsClient.getConf().getWritePacketSize(), freeInLastBlock),
-          bytesPerChecksum);
-    }
-  }
-
-  static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
-      EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
-      LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
-      String[] favoredNodes) throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("newStreamForAppend", src);
-    try {
-      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
-          progress, lastBlock, stat, checksum, favoredNodes);
-      out.start();
-      return out;
-    } finally {
-      scope.close();
-    }
-  }
-
-  protected void computePacketChunkSize(int psize, int csize) {
-    final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
-    final int chunkSize = csize + getChecksumSize();
-    chunksPerPacket = Math.max(bodySize/chunkSize, 1);
-    packetSize = chunkSize*chunksPerPacket;
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("computePacketChunkSize: src=" + src +
-                ", chunkSize=" + chunkSize +
-                ", chunksPerPacket=" + chunksPerPacket +
-                ", packetSize=" + packetSize);
-    }
-  }
-
-  protected TraceScope createWriteTraceScope() {
-    return dfsClient.getPathTraceScope("DFSOutputStream#write", src);
-  }
-
-  // @see FSOutputSummer#writeChunk()
-  @Override
-  protected synchronized void writeChunk(byte[] b, int offset, int len,
-      byte[] checksum, int ckoff, int cklen) throws IOException {
-    dfsClient.checkOpen();
-    checkClosed();
-
-    if (len > bytesPerChecksum) {
-      throw new IOException("writeChunk() buffer size is " + len +
-                            " is larger than supported  bytesPerChecksum " +
-                            bytesPerChecksum);
-    }
-    if (cklen != 0 && cklen != getChecksumSize()) {
-      throw new IOException("writeChunk() checksum size is supposed to be " +
-                            getChecksumSize() + " but found to be " + cklen);
-    }
-
-    if (currentPacket == null) {
-      currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
-          .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + 
-            currentPacket.getSeqno() +
-            ", src=" + src +
-            ", packetSize=" + packetSize +
-            ", chunksPerPacket=" + chunksPerPacket +
-            ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
-      }
-    }
-
-    currentPacket.writeChecksum(checksum, ckoff, cklen);
-    currentPacket.writeData(b, offset, len);
-    currentPacket.incNumChunks();
-    getStreamer().incBytesCurBlock(len);
-
-    // If packet is full, enqueue it for transmission
-    //
-    if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
-        getStreamer().getBytesCurBlock() == blockSize) {
-      enqueueCurrentPacketFull();
-    }
-  }
-
-  void enqueueCurrentPacket() throws IOException {
-    getStreamer().waitAndQueuePacket(currentPacket);
-    currentPacket = null;
-  }
-
-  void enqueueCurrentPacketFull() throws IOException {
-    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
-        + " appendChunk={}, {}", currentPacket, src, getStreamer()
-        .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
-        getStreamer());
-    enqueueCurrentPacket();
-    adjustChunkBoundary();
-    endBlock();
-  }
-
-  /** create an empty packet to mark the end of the block. */
-  void setCurrentPacketToEmpty() throws InterruptedIOException {
-    currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
-        getStreamer().getAndIncCurrentSeqno(), true);
-    currentPacket.setSyncBlock(shouldSyncBlock);
-  }
-
-  /**
-   * If the reopened file did not end at chunk boundary and the above
-   * write filled up its partial chunk. Tell the summer to generate full
-   * crc chunks from now on.
-   */
-  protected void adjustChunkBoundary() {
-    if (getStreamer().getAppendChunk() &&
-        getStreamer().getBytesCurBlock() % bytesPerChecksum == 0) {
-      getStreamer().setAppendChunk(false);
-      resetChecksumBufSize();
-    }
-
-    if (!getStreamer().getAppendChunk()) {
-      int psize = Math.min((int)(blockSize- getStreamer().getBytesCurBlock()),
-          dfsClient.getConf().getWritePacketSize());
-      computePacketChunkSize(psize, bytesPerChecksum);
-    }
-  }
-
-  /**
-   * if encountering a block boundary, send an empty packet to
-   * indicate the end of block and reset bytesCurBlock.
-   *
-   * @throws IOException
-   */
-  protected void endBlock() throws IOException {
-    if (getStreamer().getBytesCurBlock() == blockSize) {
-      setCurrentPacketToEmpty();
-      enqueueCurrentPacket();
-      getStreamer().setBytesCurBlock(0);
-      lastFlushOffset = 0;
-    }
-  }
-
-  @Deprecated
-  public void sync() throws IOException {
-    hflush();
-  }
-  
-  /**
-   * Flushes out to all replicas of the block. The data is in the buffers
-   * of the DNs but not necessarily in the DN's OS buffers.
-   *
-   * It is a synchronous operation. When it returns,
-   * it guarantees that flushed data become visible to new readers. 
-   * It is not guaranteed that data has been flushed to 
-   * persistent store on the datanode. 
-   * Block allocations are persisted on namenode.
-   */
-  @Override
-  public void hflush() throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("hflush", src);
-    try {
-      flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
-    } finally {
-      scope.close();
-    }
-  }
-
-  @Override
-  public void hsync() throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("hsync", src);
-    try {
-      flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * The expected semantics is all data have flushed out to all replicas 
-   * and all replicas have done posix fsync equivalent - ie the OS has 
-   * flushed it to the disk device (but the disk may have it in its cache).
-   * 
-   * Note that only the current block is flushed to the disk device.
-   * To guarantee durable sync across block boundaries the stream should
-   * be created with {@link CreateFlag#SYNC_BLOCK}.
-   * 
-   * @param syncFlags
-   *          Indicate the semantic of the sync. Currently used to specify
-   *          whether or not to update the block length in NameNode.
-   */
-  public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("hsync", src);
-    try {
-      flushOrSync(true, syncFlags);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Flush/Sync buffered data to DataNodes.
-   * 
-   * @param isSync
-   *          Whether or not to require all replicas to flush data to the disk
-   *          device
-   * @param syncFlags
-   *          Indicate extra detailed semantic of the flush/sync. Currently
-   *          mainly used to specify whether or not to update the file length in
-   *          the NameNode
-   * @throws IOException
-   */
-  private void flushOrSync(boolean isSync, EnumSet<SyncFlag> syncFlags)
-      throws IOException {
-    dfsClient.checkOpen();
-    checkClosed();
-    try {
-      long toWaitFor;
-      long lastBlockLength = -1L;
-      boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
-      boolean endBlock = syncFlags.contains(SyncFlag.END_BLOCK);
-      synchronized (this) {
-        // flush checksum buffer, but keep checksum buffer intact if we do not
-        // need to end the current block
-        int numKept = flushBuffer(!endBlock, true);
-        // bytesCurBlock potentially incremented if there was buffered data
-
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("DFSClient flush(): "
-              + " bytesCurBlock=" + getStreamer().getBytesCurBlock()
-              + " lastFlushOffset=" + lastFlushOffset
-              + " createNewBlock=" + endBlock);
-        }
-        // Flush only if we haven't already flushed till this offset.
-        if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
-          assert getStreamer().getBytesCurBlock() > lastFlushOffset;
-          // record the valid offset of this flush
-          lastFlushOffset = getStreamer().getBytesCurBlock();
-          if (isSync && currentPacket == null && !endBlock) {
-            // Nothing to send right now,
-            // but sync was requested.
-            // Send an empty packet if we do not end the block right now
-            currentPacket = createPacket(packetSize, chunksPerPacket,
-                getStreamer().getBytesCurBlock(), getStreamer()
-                    .getAndIncCurrentSeqno(), false);
-          }
-        } else {
-          if (isSync && getStreamer().getBytesCurBlock() > 0 && !endBlock) {
-            // Nothing to send right now,
-            // and the block was partially written,
-            // and sync was requested.
-            // So send an empty sync packet if we do not end the block right
-            // now
-            currentPacket = createPacket(packetSize, chunksPerPacket,
-                getStreamer().getBytesCurBlock(), getStreamer()
-                    .getAndIncCurrentSeqno(), false);
-          } else if (currentPacket != null) {
-            // just discard the current packet since it is already been sent.
-            currentPacket.releaseBuffer(byteArrayManager);
-            currentPacket = null;
-          }
-        }
-        if (currentPacket != null) {
-          currentPacket.setSyncBlock(isSync);
-          enqueueCurrentPacket();
-        }
-        if (endBlock && getStreamer().getBytesCurBlock() > 0) {
-          // Need to end the current block, thus send an empty packet to
-          // indicate this is the end of the block and reset bytesCurBlock
-          currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
-              getStreamer().getAndIncCurrentSeqno(), true);
-          currentPacket.setSyncBlock(shouldSyncBlock || isSync);
-          enqueueCurrentPacket();
-          getStreamer().setBytesCurBlock(0);
-          lastFlushOffset = 0;
-        } else {
-          // Restore state of stream. Record the last flush offset
-          // of the last full chunk that was flushed.
-          getStreamer().setBytesCurBlock(
-              getStreamer().getBytesCurBlock() - numKept);
-        }
-
-        toWaitFor = getStreamer().getLastQueuedSeqno();
-      } // end synchronized
-
-      getStreamer().waitForAckedSeqno(toWaitFor);
-
-      // update the block length first time irrespective of flag
-      if (updateLength || getStreamer().getPersistBlocks().get()) {
-        synchronized (this) {
-          if (!getStreamer().streamerClosed()
-              && getStreamer().getBlock() != null) {
-            lastBlockLength = getStreamer().getBlock().getNumBytes();
-          }
-        }
-      }
-      // If 1) any new blocks were allocated since the last flush, or 2) to
-      // update length in NN is required, then persist block locations on
-      // namenode.
-      if (getStreamer().getPersistBlocks().getAndSet(false) || updateLength) {
-        try {
-          dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
-              lastBlockLength);
-        } catch (IOException ioe) {
-          DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
-          // If we got an error here, it might be because some other thread called
-          // close before our hflush completed. In that case, we should throw an
-          // exception that the stream is closed.
-          checkClosed();
-          // If we aren't closed but failed to sync, we should expose that to the
-          // caller.
-          throw ioe;
-        }
-      }
-
-      synchronized(this) {
-        if (!getStreamer().streamerClosed()) {
-          getStreamer().setHflush();
-        }
-      }
-    } catch (InterruptedIOException interrupt) {
-      // This kind of error doesn't mean that the stream itself is broken - just the
-      // flushing thread got interrupted. So, we shouldn't close down the writer,
-      // but instead just propagate the error
-      throw interrupt;
-    } catch (IOException e) {
-      DFSClient.LOG.warn("Error while syncing", e);
-      synchronized (this) {
-        if (!isClosed()) {
-          getStreamer().getLastException().set(e);
-          closeThreads(true);
-        }
-      }
-      throw e;
-    }
-  }
-
-  /**
-   * @deprecated use {@link HdfsDataOutputStream#getCurrentBlockReplication()}.
-   */
-  @Deprecated
-  public synchronized int getNumCurrentReplicas() throws IOException {
-    return getCurrentBlockReplication();
-  }
-
-  /**
-   * Note that this is not a public API;
-   * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
-   * 
-   * @return the number of valid replicas of the current block
-   */
-  public synchronized int getCurrentBlockReplication() throws IOException {
-    dfsClient.checkOpen();
-    checkClosed();
-    if (getStreamer().streamerClosed()) {
-      return blockReplication; // no pipeline, return repl factor of file
-    }
-    DatanodeInfo[] currentNodes = getStreamer().getNodes();
-    if (currentNodes == null) {
-      return blockReplication; // no pipeline, return repl factor of file
-    }
-    return currentNodes.length;
-  }
-  
-  /**
-   * Waits till all existing data is flushed and confirmations 
-   * received from datanodes. 
-   */
-  protected void flushInternal() throws IOException {
-    long toWaitFor;
-    synchronized (this) {
-      dfsClient.checkOpen();
-      checkClosed();
-      //
-      // If there is data in the current buffer, send it across
-      //
-      getStreamer().queuePacket(currentPacket);
-      currentPacket = null;
-      toWaitFor = getStreamer().getLastQueuedSeqno();
-    }
-
-    getStreamer().waitForAckedSeqno(toWaitFor);
-  }
-
-  protected synchronized void start() {
-    getStreamer().start();
-  }
-  
-  /**
-   * Aborts this output stream and releases any system 
-   * resources associated with this stream.
-   */
-  synchronized void abort() throws IOException {
-    if (isClosed()) {
-      return;
-    }
-    getStreamer().getLastException().set(new IOException("Lease timeout of "
-        + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
-    closeThreads(true);
-    dfsClient.endFileLease(fileId);
-  }
-
-  boolean isClosed() {
-    return closed || getStreamer().streamerClosed();
-  }
-
-  void setClosed() {
-    closed = true;
-    getStreamer().release();
-  }
-
-  // shutdown datastreamer and responseprocessor threads.
-  // interrupt datastreamer if force is true
-  protected void closeThreads(boolean force) throws IOException {
-    try {
-      getStreamer().close(force);
-      getStreamer().join();
-      getStreamer().closeSocket();
-    } catch (InterruptedException e) {
-      throw new IOException("Failed to shutdown streamer");
-    } finally {
-      getStreamer().setSocketToNull();
-      setClosed();
-    }
-  }
-  
-  /**
-   * Closes this output stream and releases any system 
-   * resources associated with this stream.
-   */
-  @Override
-  public synchronized void close() throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("DFSOutputStream#close", src);
-    try {
-      closeImpl();
-    } finally {
-      scope.close();
-    }
-  }
-
-  protected synchronized void closeImpl() throws IOException {
-    if (isClosed()) {
-      getStreamer().getLastException().check(true);
-      return;
-    }
-
-    try {
-      flushBuffer();       // flush from all upper layers
-
-      if (currentPacket != null) {
-        enqueueCurrentPacket();
-      }
-
-      if (getStreamer().getBytesCurBlock() != 0) {
-        setCurrentPacketToEmpty();
-      }
-
-      flushInternal();             // flush all data to Datanodes
-      // get last block before destroying the streamer
-      ExtendedBlock lastBlock = getStreamer().getBlock();
-      closeThreads(false);
-      TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
-      try {
-        completeFile(lastBlock);
-      } finally {
-        scope.close();
-      }
-      dfsClient.endFileLease(fileId);
-    } catch (ClosedChannelException e) {
-    } finally {
-      setClosed();
-    }
-  }
-
-  // should be called holding (this) lock since setTestFilename() may 
-  // be called during unit tests
-  protected void completeFile(ExtendedBlock last) throws IOException {
-    long localstart = Time.monotonicNow();
-    final DfsClientConf conf = dfsClient.getConf();
-    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
-    boolean fileComplete = false;
-    int retries = conf.getNumBlockWriteLocateFollowingRetry();
-    while (!fileComplete) {
-      fileComplete =
-          dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
-      if (!fileComplete) {
-        final int hdfsTimeout = conf.getHdfsTimeout();
-        if (!dfsClient.clientRunning
-            || (hdfsTimeout > 0
-                && localstart + hdfsTimeout < Time.monotonicNow())) {
-            String msg = "Unable to close file because dfsclient " +
-                          " was unable to contact the HDFS servers." +
-                          " clientRunning " + dfsClient.clientRunning +
-                          " hdfsTimeout " + hdfsTimeout;
-            DFSClient.LOG.info(msg);
-            throw new IOException(msg);
-        }
-        try {
-          if (retries == 0) {
-            throw new IOException("Unable to close file because the last block"
-                + " does not have enough number of replicas.");
-          }
-          retries--;
-          Thread.sleep(sleeptime);
-          sleeptime *= 2;
-          if (Time.monotonicNow() - localstart > 5000) {
-            DFSClient.LOG.info("Could not complete " + src + " retrying...");
-          }
-        } catch (InterruptedException ie) {
-          DFSClient.LOG.warn("Caught exception ", ie);
-        }
-      }
-    }
-  }
-
-  @VisibleForTesting
-  public void setArtificialSlowdown(long period) {
-    getStreamer().setArtificialSlowdown(period);
-  }
-
-  @VisibleForTesting
-  public synchronized void setChunksPerPacket(int value) {
-    chunksPerPacket = Math.min(chunksPerPacket, value);
-    packetSize = (bytesPerChecksum + getChecksumSize()) * chunksPerPacket;
-  }
-
-  /**
-   * Returns the size of a file as it was when this stream was opened
-   */
-  public long getInitialLen() {
-    return initialFileSize;
-  }
-
-  /**
-   * @return the FileEncryptionInfo for this stream, or null if not encrypted.
-   */
-  public FileEncryptionInfo getFileEncryptionInfo() {
-    return fileEncryptionInfo;
-  }
-
-  /**
-   * Returns the access token currently used by streamer, for testing only
-   */
-  synchronized Token<BlockTokenIdentifier> getBlockToken() {
-    return getStreamer().getBlockToken();
-  }
-
-  @Override
-  public void setDropBehind(Boolean dropBehind) throws IOException {
-    CachingStrategy prevStrategy, nextStrategy;
-    // CachingStrategy is immutable.  So build a new CachingStrategy with the
-    // modifications we want, and compare-and-swap it in.
-    do {
-      prevStrategy = this.cachingStrategy.get();
-      nextStrategy = new CachingStrategy.Builder(prevStrategy).
-                        setDropBehind(dropBehind).build();
-    } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
-  }
-
-  @VisibleForTesting
-  ExtendedBlock getBlock() {
-    return getStreamer().getBlock();
-  }
-
-  @VisibleForTesting
-  public long getFileId() {
-    return fileId;
-  }
-
-  /**
-   * Return the source of stream.
-   */
-  String getSrc() {
-    return src;
-  }
-
-  /**
-   * Returns the data streamer object.
-   */
-  protected DataStreamer getStreamer() {
-    return streamer;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
deleted file mode 100755
index 22055c3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.BufferOverflowException;
-import java.nio.channels.ClosedChannelException;
-import java.util.Arrays;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.htrace.Span;
-
-/****************************************************************
- * DFSPacket is used by DataStreamer and DFSOutputStream.
- * DFSOutputStream generates packets and then ask DatStreamer
- * to send them to datanodes.
- ****************************************************************/
-
-@InterfaceAudience.Private
-class DFSPacket {
-  public static final long HEART_BEAT_SEQNO = -1L;
-  private static long[] EMPTY = new long[0];
-  private final long seqno; // sequence number of buffer in block
-  private final long offsetInBlock; // offset in block
-  private boolean syncBlock; // this packet forces the current block to disk
-  private int numChunks; // number of chunks currently in packet
-  private final int maxChunks; // max chunks in packet
-  private byte[] buf;
-  private final boolean lastPacketInBlock; // is this the last packet in block?
-
-  /**
-   * buf is pointed into like follows:
-   *  (C is checksum data, D is payload data)
-   *
-   * [_________CCCCCCCCC________________DDDDDDDDDDDDDDDD___]
-   *           ^        ^               ^               ^
-   *           |        checksumPos     dataStart       dataPos
-   *           checksumStart
-   *
-   * Right before sending, we move the checksum data to immediately precede
-   * the actual data, and then insert the header into the buffer immediately
-   * preceding the checksum data, so we make sure to keep enough space in
-   * front of the checksum data to support the largest conceivable header.
-   */
-  private int checksumStart;
-  private int checksumPos;
-  private final int dataStart;
-  private int dataPos;
-  private long[] traceParents = EMPTY;
-  private int traceParentsUsed;
-  private Span span;
-
-  /**
-   * Create a new packet.
-   *
-   * @param buf the buffer storing data and checksums
-   * @param chunksPerPkt maximum number of chunks per packet.
-   * @param offsetInBlock offset in bytes into the HDFS block.
-   * @param seqno the sequence number of this packet
-   * @param checksumSize the size of checksum
-   * @param lastPacketInBlock if this is the last packet
-   */
-  DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
-                   int checksumSize, boolean lastPacketInBlock) {
-    this.lastPacketInBlock = lastPacketInBlock;
-    this.numChunks = 0;
-    this.offsetInBlock = offsetInBlock;
-    this.seqno = seqno;
-
-    this.buf = buf;
-
-    checksumStart = PacketHeader.PKT_MAX_HEADER_LEN;
-    checksumPos = checksumStart;
-    dataStart = checksumStart + (chunksPerPkt * checksumSize);
-    dataPos = dataStart;
-    maxChunks = chunksPerPkt;
-  }
-
-  /**
-   * Write data to this packet.
-   *
-   * @param inarray input array of data
-   * @param off the offset of data to write
-   * @param len the length of data to write
-   * @throws ClosedChannelException
-   */
-  synchronized void writeData(byte[] inarray, int off, int len)
-      throws ClosedChannelException {
-    checkBuffer();
-    if (dataPos + len > buf.length) {
-      throw new BufferOverflowException();
-    }
-    System.arraycopy(inarray, off, buf, dataPos, len);
-    dataPos += len;
-  }
-
-  /**
-   * Write checksums to this packet
-   *
-   * @param inarray input array of checksums
-   * @param off the offset of checksums to write
-   * @param len the length of checksums to write
-   * @throws ClosedChannelException
-   */
-  synchronized void writeChecksum(byte[] inarray, int off, int len)
-      throws ClosedChannelException {
-    checkBuffer();
-    if (len == 0) {
-      return;
-    }
-    if (checksumPos + len > dataStart) {
-      throw new BufferOverflowException();
-    }
-    System.arraycopy(inarray, off, buf, checksumPos, len);
-    checksumPos += len;
-  }
-
-  /**
-   * Write the full packet, including the header, to the given output stream.
-   *
-   * @param stm
-   * @throws IOException
-   */
-  synchronized void writeTo(DataOutputStream stm) throws IOException {
-    checkBuffer();
-
-    final int dataLen = dataPos - dataStart;
-    final int checksumLen = checksumPos - checksumStart;
-    final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen;
-
-    PacketHeader header = new PacketHeader(
-        pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
-
-    if (checksumPos != dataStart) {
-      // Move the checksum to cover the gap. This can happen for the last
-      // packet or during an hflush/hsync call.
-      System.arraycopy(buf, checksumStart, buf,
-          dataStart - checksumLen , checksumLen);
-      checksumPos = dataStart;
-      checksumStart = checksumPos - checksumLen;
-    }
-
-    final int headerStart = checksumStart - header.getSerializedSize();
-    assert checksumStart + 1 >= header.getSerializedSize();
-    assert headerStart >= 0;
-    assert headerStart + header.getSerializedSize() == checksumStart;
-
-    // Copy the header data into the buffer immediately preceding the checksum
-    // data.
-    System.arraycopy(header.getBytes(), 0, buf, headerStart,
-        header.getSerializedSize());
-
-    // corrupt the data for testing.
-    if (DFSClientFaultInjector.get().corruptPacket()) {
-      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
-    }
-
-    // Write the now contiguous full packet to the output stream.
-    stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
-
-    // undo corruption.
-    if (DFSClientFaultInjector.get().uncorruptPacket()) {
-      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
-    }
-  }
-
-  private synchronized void checkBuffer() throws ClosedChannelException {
-    if (buf == null) {
-      throw new ClosedChannelException();
-    }
-  }
-
-  /**
-   * Release the buffer in this packet to ByteArrayManager.
-   *
-   * @param bam
-   */
-  synchronized void releaseBuffer(ByteArrayManager bam) {
-    bam.release(buf);
-    buf = null;
-  }
-
-  /**
-   * get the packet's last byte's offset in the block
-   *
-   * @return the packet's last byte's offset in the block
-   */
-  synchronized long getLastByteOffsetBlock() {
-    return offsetInBlock + dataPos - dataStart;
-  }
-
-  /**
-   * Check if this packet is a heart beat packet
-   *
-   * @return true if the sequence number is HEART_BEAT_SEQNO
-   */
-  boolean isHeartbeatPacket() {
-    return seqno == HEART_BEAT_SEQNO;
-  }
-
-  /**
-   * check if this packet is the last packet in block
-   *
-   * @return true if the packet is the last packet
-   */
-  boolean isLastPacketInBlock(){
-    return lastPacketInBlock;
-  }
-
-  /**
-   * get sequence number of this packet
-   *
-   * @return the sequence number of this packet
-   */
-  long getSeqno(){
-    return seqno;
-  }
-
-  /**
-   * get the number of chunks this packet contains
-   *
-   * @return the number of chunks in this packet
-   */
-  synchronized int getNumChunks(){
-    return numChunks;
-  }
-
-  /**
-   * increase the number of chunks by one
-   */
-  synchronized void incNumChunks(){
-    numChunks++;
-  }
-
-  /**
-   * get the maximum number of packets
-   *
-   * @return the maximum number of packets
-   */
-  int getMaxChunks(){
-    return maxChunks;
-  }
-
-  /**
-   * set if to sync block
-   *
-   * @param syncBlock if to sync block
-   */
-  synchronized void setSyncBlock(boolean syncBlock){
-    this.syncBlock = syncBlock;
-  }
-
-  @Override
-  public String toString() {
-    return "packet seqno: " + this.seqno +
-        " offsetInBlock: " + this.offsetInBlock +
-        " lastPacketInBlock: " + this.lastPacketInBlock +
-        " lastByteOffsetInBlock: " + this.getLastByteOffsetBlock();
-  }
-
-  /**
-   * Add a trace parent span for this packet.<p/>
-   *
-   * Trace parent spans for a packet are the trace spans responsible for
-   * adding data to that packet.  We store them as an array of longs for
-   * efficiency.<p/>
-   *
-   * Protected by the DFSOutputStream dataQueue lock.
-   */
-  public void addTraceParent(Span span) {
-    if (span == null) {
-      return;
-    }
-    addTraceParent(span.getSpanId());
-  }
-
-  public void addTraceParent(long id) {
-    if (traceParentsUsed == traceParents.length) {
-      int newLength = (traceParents.length == 0) ? 8 :
-          traceParents.length * 2;
-      traceParents = Arrays.copyOf(traceParents, newLength);
-    }
-    traceParents[traceParentsUsed] = id;
-    traceParentsUsed++;
-  }
-
-  /**
-   * Get the trace parent spans for this packet.<p/>
-   *
-   * Will always be non-null.<p/>
-   *
-   * Protected by the DFSOutputStream dataQueue lock.
-   */
-  public long[] getTraceParents() {
-    // Remove duplicates from the array.
-    int len = traceParentsUsed;
-    Arrays.sort(traceParents, 0, len);
-    int i = 0, j = 0;
-    long prevVal = 0; // 0 is not a valid span id
-    while (true) {
-      if (i == len) {
-        break;
-      }
-      long val = traceParents[i];
-      if (val != prevVal) {
-        traceParents[j] = val;
-        j++;
-        prevVal = val;
-      }
-      i++;
-    }
-    if (j < traceParents.length) {
-      traceParents = Arrays.copyOf(traceParents, j);
-      traceParentsUsed = traceParents.length;
-    }
-    return traceParents;
-  }
-
-  public void setTraceSpan(Span span) {
-    this.span = span;
-  }
-
-  public Span getTraceSpan() {
-    return span;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index fee7138..16695fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -1477,27 +1477,4 @@ public class DFSUtil {
     return cryptoProvider;
   }
 
-  public static int getIoFileBufferSize(Configuration conf) {
-    return conf.getInt(
-      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
-      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
-  }
-
-  public static int getSmallBufferSize(Configuration conf) {
-    return Math.min(getIoFileBufferSize(conf) / 2, 512);
-  }
-
-  /**
-   * Probe for HDFS Encryption being enabled; this uses the value of
-   * the option {@link DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI},
-   * returning true if that property contains a non-empty, non-whitespace
-   * string.
-   * @param conf configuration to probe
-   * @return true if encryption is considered enabled.
-   */
-  public static boolean isHDFSEncryptionEnabled(Configuration conf) {
-    return !conf.getTrimmed(
-        DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty();
-  }
-
 }


[10/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
new file mode 100644
index 0000000..12496e2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Used for injecting faults in DFSClient and DFSOutputStream tests.
+ * Calls into this are a no-op in production code. 
+ */
+@VisibleForTesting
+@InterfaceAudience.Private
+public class DFSClientFaultInjector {
+  private static DFSClientFaultInjector instance = new DFSClientFaultInjector();
+  public static AtomicLong exceptionNum = new AtomicLong(0);
+
+  public static DFSClientFaultInjector get() {
+    return instance;
+  }
+  public static void set(DFSClientFaultInjector instance) {
+    DFSClientFaultInjector.instance = instance;
+  }
+
+  public boolean corruptPacket() {
+    return false;
+  }
+
+  public boolean uncorruptPacket() {
+    return false;
+  }
+
+  public boolean failPacket() {
+    return false;
+  }
+
+  public void startFetchFromDatanode() {}
+
+  public void fetchFromDatanodeException() {}
+
+  public void readFromDatanodeDelay() {}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
new file mode 100644
index 0000000..2a228e8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * The client-side metrics for hedged read feature.
+ * This class has a number of metrics variables that are publicly accessible,
+ * we can grab them from client side, like HBase.
+ */
+@InterfaceAudience.Private
+public class DFSHedgedReadMetrics {
+  public final AtomicLong hedgedReadOps = new AtomicLong();
+  public final AtomicLong hedgedReadOpsWin = new AtomicLong();
+  public final AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
+
+  public void incHedgedReadOps() {
+    hedgedReadOps.incrementAndGet();
+  }
+
+  public void incHedgedReadOpsInCurThread() {
+    hedgedReadOpsInCurThread.incrementAndGet();
+  }
+
+  public void incHedgedReadWins() {
+    hedgedReadOpsWin.incrementAndGet();
+  }
+
+  public long getHedgedReadOps() {
+    return hedgedReadOps.longValue();
+  }
+
+  public long getHedgedReadOpsInCurThread() {
+    return hedgedReadOpsInCurThread.longValue();
+  }
+
+  public long getHedgedReadWins() {
+    return hedgedReadOpsWin.longValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
new file mode 100644
index 0000000..11a1d29
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
@@ -0,0 +1,239 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import com.google.common.collect.Iterators;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.inotify.EventBatch;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.inotify.MissingEventsException;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Stream for reading inotify events. DFSInotifyEventInputStreams should not
+ * be shared among multiple threads.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class DFSInotifyEventInputStream {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      DFSInotifyEventInputStream.class);
+
+  /**
+   * The trace sampler to use when making RPCs to the NameNode.
+   */
+  private final Sampler<?> traceSampler;
+
+  private final ClientProtocol namenode;
+  private Iterator<EventBatch> it;
+  private long lastReadTxid;
+  /**
+   * The most recent txid the NameNode told us it has sync'ed -- helps us
+   * determine how far behind we are in the edit stream.
+   */
+  private long syncTxid;
+  /**
+   * Used to generate wait times in {@link DFSInotifyEventInputStream#take()}.
+   */
+  private Random rng = new Random();
+
+  private static final int INITIAL_WAIT_MS = 10;
+
+  DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode)
+        throws IOException {
+    // Only consider new transaction IDs.
+    this(traceSampler, namenode, namenode.getCurrentEditLogTxid());
+  }
+
+  DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode,
+        long lastReadTxid) throws IOException {
+    this.traceSampler = traceSampler;
+    this.namenode = namenode;
+    this.it = Iterators.emptyIterator();
+    this.lastReadTxid = lastReadTxid;
+  }
+
+  /**
+   * Returns the next batch of events in the stream or null if no new
+   * batches are currently available.
+   *
+   * @throws IOException because of network error or edit log
+   * corruption. Also possible if JournalNodes are unresponsive in the
+   * QJM setting (even one unresponsive JournalNode is enough in rare cases),
+   * so catching this exception and retrying at least a few times is
+   * recommended.
+   * @throws MissingEventsException if we cannot return the next batch in the
+   * stream because the data for the events (and possibly some subsequent
+   * events) has been deleted (generally because this stream is a very large
+   * number of transactions behind the current state of the NameNode). It is
+   * safe to continue reading from the stream after this exception is thrown
+   * The next available batch of events will be returned.
+   */
+  public EventBatch poll() throws IOException, MissingEventsException {
+    TraceScope scope =
+        Trace.startSpan("inotifyPoll", traceSampler);
+    try {
+      // need to keep retrying until the NN sends us the latest committed txid
+      if (lastReadTxid == -1) {
+        LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
+        lastReadTxid = namenode.getCurrentEditLogTxid();
+        return null;
+      }
+      if (!it.hasNext()) {
+        EventBatchList el = namenode.getEditsFromTxid(lastReadTxid + 1);
+        if (el.getLastTxid() != -1) {
+          // we only want to set syncTxid when we were actually able to read some
+          // edits on the NN -- otherwise it will seem like edits are being
+          // generated faster than we can read them when the problem is really
+          // that we are temporarily unable to read edits
+          syncTxid = el.getSyncTxid();
+          it = el.getBatches().iterator();
+          long formerLastReadTxid = lastReadTxid;
+          lastReadTxid = el.getLastTxid();
+          if (el.getFirstTxid() != formerLastReadTxid + 1) {
+            throw new MissingEventsException(formerLastReadTxid + 1,
+                el.getFirstTxid());
+          }
+        } else {
+          LOG.debug("poll(): read no edits from the NN when requesting edits " +
+            "after txid {}", lastReadTxid);
+          return null;
+        }
+      }
+
+      if (it.hasNext()) { // can be empty if el.getLastTxid != -1 but none of the
+        // newly seen edit log ops actually got converted to events
+        return it.next();
+      } else {
+        return null;
+      }
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Return a estimate of how many transaction IDs behind the NameNode's
+   * current state this stream is. Clients should periodically call this method
+   * and check if its result is steadily increasing, which indicates that they
+   * are falling behind (i.e. transaction are being generated faster than the
+   * client is reading them). If a client falls too far behind events may be
+   * deleted before the client can read them.
+   * <p/>
+   * A return value of -1 indicates that an estimate could not be produced, and
+   * should be ignored. The value returned by this method is really only useful
+   * when compared to previous or subsequent returned values.
+   */
+  public long getTxidsBehindEstimate() {
+    if (syncTxid == 0) {
+      return -1;
+    } else {
+      assert syncTxid >= lastReadTxid;
+      // this gives the difference between the last txid we have fetched to the
+      // client and syncTxid at the time we last fetched events from the
+      // NameNode
+      return syncTxid - lastReadTxid;
+    }
+  }
+
+  /**
+   * Returns the next event batch in the stream, waiting up to the specified
+   * amount of time for a new batch. Returns null if one is not available at the
+   * end of the specified amount of time. The time before the method returns may
+   * exceed the specified amount of time by up to the time required for an RPC
+   * to the NameNode.
+   *
+   * @param time number of units of the given TimeUnit to wait
+   * @param tu the desired TimeUnit
+   * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
+   * @throws MissingEventsException
+   * see {@link DFSInotifyEventInputStream#poll()}
+   * @throws InterruptedException if the calling thread is interrupted
+   */
+  public EventBatch poll(long time, TimeUnit tu) throws IOException,
+      InterruptedException, MissingEventsException {
+    TraceScope scope = Trace.startSpan("inotifyPollWithTimeout", traceSampler);
+    EventBatch next = null;
+    try {
+      long initialTime = Time.monotonicNow();
+      long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
+      long nextWait = INITIAL_WAIT_MS;
+      while ((next = poll()) == null) {
+        long timeLeft = totalWait - (Time.monotonicNow() - initialTime);
+        if (timeLeft <= 0) {
+          LOG.debug("timed poll(): timed out");
+          break;
+        } else if (timeLeft < nextWait * 2) {
+          nextWait = timeLeft;
+        } else {
+          nextWait *= 2;
+        }
+        LOG.debug("timed poll(): poll() returned null, sleeping for {} ms",
+            nextWait);
+        Thread.sleep(nextWait);
+      }
+    } finally {
+      scope.close();
+    }
+    return next;
+  }
+
+  /**
+   * Returns the next batch of events in the stream, waiting indefinitely if
+   * a new batch  is not immediately available.
+   *
+   * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
+   * @throws MissingEventsException see
+   * {@link DFSInotifyEventInputStream#poll()}
+   * @throws InterruptedException if the calling thread is interrupted
+   */
+  public EventBatch take() throws IOException, InterruptedException,
+      MissingEventsException {
+    TraceScope scope = Trace.startSpan("inotifyTake", traceSampler);
+    EventBatch next = null;
+    try {
+      int nextWaitMin = INITIAL_WAIT_MS;
+      while ((next = poll()) == null) {
+        // sleep for a random period between nextWaitMin and nextWaitMin * 2
+        // to avoid stampedes at the NN if there are multiple clients
+        int sleepTime = nextWaitMin + rng.nextInt(nextWaitMin);
+        LOG.debug("take(): poll() returned null, sleeping for {} ms", sleepTime);
+        Thread.sleep(sleepTime);
+        // the maximum sleep is 2 minutes
+        nextWaitMin = Math.min(60000, nextWaitMin * 2);
+      }
+    } finally {
+      scope.close();
+    }
+
+    return next;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
new file mode 100644
index 0000000..139a27c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -0,0 +1,1915 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.ByteBufferUtil;
+import org.apache.hadoop.fs.CanSetDropBehind;
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.CanUnbuffer;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
+import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
+import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.IdentityHashStore;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/****************************************************************
+ * DFSInputStream provides bytes from a named file.  It handles 
+ * negotiation of the namenode and various datanodes as necessary.
+ ****************************************************************/
+@InterfaceAudience.Private
+public class DFSInputStream extends FSInputStream
+implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
+    HasEnhancedByteBufferAccess, CanUnbuffer {
+  @VisibleForTesting
+  public static boolean tcpReadsDisabledForTesting = false;
+  private long hedgedReadOpsLoopNumForTesting = 0;
+  protected final DFSClient dfsClient;
+  protected AtomicBoolean closed = new AtomicBoolean(false);
+  protected final String src;
+  protected final boolean verifyChecksum;
+
+  // state by stateful read only:
+  // (protected by lock on this)
+  /////
+  private DatanodeInfo currentNode = null;
+  protected LocatedBlock currentLocatedBlock = null;
+  protected long pos = 0;
+  protected long blockEnd = -1;
+  private BlockReader blockReader = null;
+  ////
+
+  // state shared by stateful and positional read:
+  // (protected by lock on infoLock)
+  ////
+  protected LocatedBlocks locatedBlocks = null;
+  private long lastBlockBeingWrittenLength = 0;
+  private FileEncryptionInfo fileEncryptionInfo = null;
+  protected CachingStrategy cachingStrategy;
+  ////
+
+  protected final ReadStatistics readStatistics = new ReadStatistics();
+  // lock for state shared between read and pread
+  // Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
+  //       (it's OK to acquire this lock when the lock on <this> is held)
+  protected final Object infoLock = new Object();
+
+  /**
+   * Track the ByteBuffers that we have handed out to readers.
+   * 
+   * The value type can be either ByteBufferPool or ClientMmap, depending on
+   * whether we this is a memory-mapped buffer or not.
+   */
+  private IdentityHashStore<ByteBuffer, Object> extendedReadBuffers;
+
+  private synchronized IdentityHashStore<ByteBuffer, Object>
+        getExtendedReadBuffers() {
+    if (extendedReadBuffers == null) {
+      extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
+    }
+    return extendedReadBuffers;
+  }
+
+  public static class ReadStatistics {
+    public ReadStatistics() {
+      clear();
+    }
+
+    public ReadStatistics(ReadStatistics rhs) {
+      this.totalBytesRead = rhs.getTotalBytesRead();
+      this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
+      this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
+      this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
+    }
+
+    /**
+     * @return The total bytes read.  This will always be at least as
+     * high as the other numbers, since it includes all of them.
+     */
+    public long getTotalBytesRead() {
+      return totalBytesRead;
+    }
+
+    /**
+     * @return The total local bytes read.  This will always be at least
+     * as high as totalShortCircuitBytesRead, since all short-circuit
+     * reads are also local.
+     */
+    public long getTotalLocalBytesRead() {
+      return totalLocalBytesRead;
+    }
+
+    /**
+     * @return The total short-circuit local bytes read.
+     */
+    public long getTotalShortCircuitBytesRead() {
+      return totalShortCircuitBytesRead;
+    }
+    
+    /**
+     * @return The total number of zero-copy bytes read.
+     */
+    public long getTotalZeroCopyBytesRead() {
+      return totalZeroCopyBytesRead;
+    }
+
+    /**
+     * @return The total number of bytes read which were not local.
+     */
+    public long getRemoteBytesRead() {
+      return totalBytesRead - totalLocalBytesRead;
+    }
+    
+    void addRemoteBytes(long amt) {
+      this.totalBytesRead += amt;
+    }
+
+    void addLocalBytes(long amt) {
+      this.totalBytesRead += amt;
+      this.totalLocalBytesRead += amt;
+    }
+
+    void addShortCircuitBytes(long amt) {
+      this.totalBytesRead += amt;
+      this.totalLocalBytesRead += amt;
+      this.totalShortCircuitBytesRead += amt;
+    }
+
+    void addZeroCopyBytes(long amt) {
+      this.totalBytesRead += amt;
+      this.totalLocalBytesRead += amt;
+      this.totalShortCircuitBytesRead += amt;
+      this.totalZeroCopyBytesRead += amt;
+    }
+
+    void clear() {
+      this.totalBytesRead = 0;
+      this.totalLocalBytesRead = 0;
+      this.totalShortCircuitBytesRead = 0;
+      this.totalZeroCopyBytesRead = 0;
+    }
+    
+    private long totalBytesRead;
+
+    private long totalLocalBytesRead;
+
+    private long totalShortCircuitBytesRead;
+
+    private long totalZeroCopyBytesRead;
+  }
+  
+  /**
+   * This variable tracks the number of failures since the start of the
+   * most recent user-facing operation. That is to say, it should be reset
+   * whenever the user makes a call on this stream, and if at any point
+   * during the retry logic, the failure count exceeds a threshold,
+   * the errors will be thrown back to the operation.
+   *
+   * Specifically this counts the number of times the client has gone
+   * back to the namenode to get a new list of block locations, and is
+   * capped at maxBlockAcquireFailures
+   */
+  protected int failures = 0;
+
+  /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
+   * parallel accesses to DFSInputStream (through ptreads) properly */
+  private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
+             new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
+
+  private byte[] oneByteBuf; // used for 'int read()'
+
+  void addToDeadNodes(DatanodeInfo dnInfo) {
+    deadNodes.put(dnInfo, dnInfo);
+  }
+  
+  DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
+      LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
+    this.dfsClient = dfsClient;
+    this.verifyChecksum = verifyChecksum;
+    this.src = src;
+    synchronized (infoLock) {
+      this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
+    }
+    this.locatedBlocks = locatedBlocks;
+    openInfo(false);
+  }
+
+  /**
+   * Grab the open-file info from namenode
+   * @param refreshLocatedBlocks whether to re-fetch locatedblocks
+   */
+  void openInfo(boolean refreshLocatedBlocks) throws IOException,
+      UnresolvedLinkException {
+    final DfsClientConf conf = dfsClient.getConf();
+    synchronized(infoLock) {
+      lastBlockBeingWrittenLength =
+          fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
+      int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
+      while (retriesForLastBlockLength > 0) {
+        // Getting last block length as -1 is a special case. When cluster
+        // restarts, DNs may not report immediately. At this time partial block
+        // locations will not be available with NN for getting the length. Lets
+        // retry for 3 times to get the length.
+        if (lastBlockBeingWrittenLength == -1) {
+          DFSClient.LOG.warn("Last block locations not available. "
+              + "Datanodes might not have reported blocks completely."
+              + " Will retry for " + retriesForLastBlockLength + " times");
+          waitFor(conf.getRetryIntervalForGetLastBlockLength());
+          lastBlockBeingWrittenLength =
+              fetchLocatedBlocksAndGetLastBlockLength(true);
+        } else {
+          break;
+        }
+        retriesForLastBlockLength--;
+      }
+      if (retriesForLastBlockLength == 0) {
+        throw new IOException("Could not obtain the last block locations.");
+      }
+    }
+  }
+
+  private void waitFor(int waitTime) throws IOException {
+    try {
+      Thread.sleep(waitTime);
+    } catch (InterruptedException e) {
+      throw new IOException(
+          "Interrupted while getting the last block length.");
+    }
+  }
+
+  private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
+      throws IOException {
+    LocatedBlocks newInfo = locatedBlocks;
+    if (locatedBlocks == null || refresh) {
+      newInfo = dfsClient.getLocatedBlocks(src, 0);
+    }
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("newInfo = " + newInfo);
+    }
+    if (newInfo == null) {
+      throw new IOException("Cannot open filename " + src);
+    }
+
+    if (locatedBlocks != null) {
+      Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
+      Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
+      while (oldIter.hasNext() && newIter.hasNext()) {
+        if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
+          throw new IOException("Blocklist for " + src + " has changed!");
+        }
+      }
+    }
+    locatedBlocks = newInfo;
+    long lastBlockBeingWrittenLength = 0;
+    if (!locatedBlocks.isLastBlockComplete()) {
+      final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
+      if (last != null) {
+        if (last.getLocations().length == 0) {
+          if (last.getBlockSize() == 0) {
+            // if the length is zero, then no data has been written to
+            // datanode. So no need to wait for the locations.
+            return 0;
+          }
+          return -1;
+        }
+        final long len = readBlockLength(last);
+        last.getBlock().setNumBytes(len);
+        lastBlockBeingWrittenLength = len; 
+      }
+    }
+
+    fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
+
+    return lastBlockBeingWrittenLength;
+  }
+
+  /** Read the block length from one of the datanodes. */
+  private long readBlockLength(LocatedBlock locatedblock) throws IOException {
+    assert locatedblock != null : "LocatedBlock cannot be null";
+    int replicaNotFoundCount = locatedblock.getLocations().length;
+    
+    final DfsClientConf conf = dfsClient.getConf();
+    for(DatanodeInfo datanode : locatedblock.getLocations()) {
+      ClientDatanodeProtocol cdp = null;
+      
+      try {
+        cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
+            dfsClient.getConfiguration(), conf.getSocketTimeout(),
+            conf.isConnectToDnViaHostname(), locatedblock);
+        
+        final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
+        
+        if (n >= 0) {
+          return n;
+        }
+      }
+      catch(IOException ioe) {
+        if (ioe instanceof RemoteException &&
+          (((RemoteException) ioe).unwrapRemoteException() instanceof
+            ReplicaNotFoundException)) {
+          // special case : replica might not be on the DN, treat as 0 length
+          replicaNotFoundCount--;
+        }
+        
+        if (DFSClient.LOG.isDebugEnabled()) {
+          DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
+              + datanode + " for block " + locatedblock.getBlock(), ioe);
+        }
+      } finally {
+        if (cdp != null) {
+          RPC.stopProxy(cdp);
+        }
+      }
+    }
+
+    // Namenode told us about these locations, but none know about the replica
+    // means that we hit the race between pipeline creation start and end.
+    // we require all 3 because some other exception could have happened
+    // on a DN that has it.  we want to report that error
+    if (replicaNotFoundCount == 0) {
+      return 0;
+    }
+
+    throw new IOException("Cannot obtain block length for " + locatedblock);
+  }
+  
+  public long getFileLength() {
+    synchronized(infoLock) {
+      return locatedBlocks == null? 0:
+          locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
+    }
+  }
+
+  // Short circuit local reads are forbidden for files that are
+  // under construction.  See HDFS-2757.
+  boolean shortCircuitForbidden() {
+    synchronized(infoLock) {
+      return locatedBlocks.isUnderConstruction();
+    }
+  }
+
+  /**
+   * Returns the datanode from which the stream is currently reading.
+   */
+  public synchronized DatanodeInfo getCurrentDatanode() {
+    return currentNode;
+  }
+
+  /**
+   * Returns the block containing the target position. 
+   */
+  synchronized public ExtendedBlock getCurrentBlock() {
+    if (currentLocatedBlock == null){
+      return null;
+    }
+    return currentLocatedBlock.getBlock();
+  }
+
+  /**
+   * Return collection of blocks that has already been located.
+   */
+  public List<LocatedBlock> getAllBlocks() throws IOException {
+    return getBlockRange(0, getFileLength());
+  }
+
+  /**
+   * Get block at the specified position.
+   * Fetch it from the namenode if not cached.
+   * 
+   * @param offset block corresponding to this offset in file is returned
+   * @return located block
+   * @throws IOException
+   */
+  protected LocatedBlock getBlockAt(long offset) throws IOException {
+    synchronized(infoLock) {
+      assert (locatedBlocks != null) : "locatedBlocks is null";
+
+      final LocatedBlock blk;
+
+      //check offset
+      if (offset < 0 || offset >= getFileLength()) {
+        throw new IOException("offset < 0 || offset >= getFileLength(), offset="
+            + offset
+            + ", locatedBlocks=" + locatedBlocks);
+      }
+      else if (offset >= locatedBlocks.getFileLength()) {
+        // offset to the portion of the last block,
+        // which is not known to the name-node yet;
+        // getting the last block
+        blk = locatedBlocks.getLastLocatedBlock();
+      }
+      else {
+        // search cached blocks first
+        int targetBlockIdx = locatedBlocks.findBlock(offset);
+        if (targetBlockIdx < 0) { // block is not cached
+          targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+          // fetch more blocks
+          final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
+          assert (newBlocks != null) : "Could not find target position " + offset;
+          locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+        }
+        blk = locatedBlocks.get(targetBlockIdx);
+      }
+      return blk;
+    }
+  }
+
+  /** Fetch a block from namenode and cache it */
+  protected void fetchBlockAt(long offset) throws IOException {
+    synchronized(infoLock) {
+      int targetBlockIdx = locatedBlocks.findBlock(offset);
+      if (targetBlockIdx < 0) { // block is not cached
+        targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+      }
+      // fetch blocks
+      final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
+      if (newBlocks == null) {
+        throw new IOException("Could not find target position " + offset);
+      }
+      locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+    }
+  }
+
+  /**
+   * Get blocks in the specified range.
+   * Fetch them from the namenode if not cached. This function
+   * will not get a read request beyond the EOF.
+   * @param offset starting offset in file
+   * @param length length of data
+   * @return consequent segment of located blocks
+   * @throws IOException
+   */
+  private List<LocatedBlock> getBlockRange(long offset,
+      long length)  throws IOException {
+    // getFileLength(): returns total file length
+    // locatedBlocks.getFileLength(): returns length of completed blocks
+    if (offset >= getFileLength()) {
+      throw new IOException("Offset: " + offset +
+        " exceeds file length: " + getFileLength());
+    }
+    synchronized(infoLock) {
+      final List<LocatedBlock> blocks;
+      final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
+      final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
+      final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
+
+      if (readOffsetWithinCompleteBlk) {
+        //get the blocks of finalized (completed) block range
+        blocks = getFinalizedBlockRange(offset,
+          Math.min(length, lengthOfCompleteBlk - offset));
+      } else {
+        blocks = new ArrayList<LocatedBlock>(1);
+      }
+
+      // get the blocks from incomplete block range
+      if (readLengthPastCompleteBlk) {
+         blocks.add(locatedBlocks.getLastLocatedBlock());
+      }
+
+      return blocks;
+    }
+  }
+
+  /**
+   * Get blocks in the specified range.
+   * Includes only the complete blocks.
+   * Fetch them from the namenode if not cached.
+   */
+  private List<LocatedBlock> getFinalizedBlockRange(
+      long offset, long length) throws IOException {
+    synchronized(infoLock) {
+      assert (locatedBlocks != null) : "locatedBlocks is null";
+      List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
+      // search cached blocks first
+      int blockIdx = locatedBlocks.findBlock(offset);
+      if (blockIdx < 0) { // block is not cached
+        blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
+      }
+      long remaining = length;
+      long curOff = offset;
+      while(remaining > 0) {
+        LocatedBlock blk = null;
+        if(blockIdx < locatedBlocks.locatedBlockCount())
+          blk = locatedBlocks.get(blockIdx);
+        if (blk == null || curOff < blk.getStartOffset()) {
+          LocatedBlocks newBlocks;
+          newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
+          locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
+          continue;
+        }
+        assert curOff >= blk.getStartOffset() : "Block not found";
+        blockRange.add(blk);
+        long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
+        remaining -= bytesRead;
+        curOff += bytesRead;
+        blockIdx++;
+      }
+      return blockRange;
+    }
+  }
+
+  /**
+   * Open a DataInputStream to a DataNode so that it can be read from.
+   * We get block ID and the IDs of the destinations at startup, from the namenode.
+   */
+  private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
+    if (target >= getFileLength()) {
+      throw new IOException("Attempted to read past end of file");
+    }
+
+    // Will be getting a new BlockReader.
+    closeCurrentBlockReaders();
+
+    //
+    // Connect to best DataNode for desired Block, with potential offset
+    //
+    DatanodeInfo chosenNode = null;
+    int refetchToken = 1; // only need to get a new access token once
+    int refetchEncryptionKey = 1; // only need to get a new encryption key once
+    
+    boolean connectFailedOnce = false;
+
+    while (true) {
+      //
+      // Compute desired block
+      //
+      LocatedBlock targetBlock = getBlockAt(target);
+
+      // update current position
+      this.pos = target;
+      this.blockEnd = targetBlock.getStartOffset() +
+            targetBlock.getBlockSize() - 1;
+      this.currentLocatedBlock = targetBlock;
+
+      long offsetIntoBlock = target - targetBlock.getStartOffset();
+
+      DNAddrPair retval = chooseDataNode(targetBlock, null);
+      chosenNode = retval.info;
+      InetSocketAddress targetAddr = retval.addr;
+      StorageType storageType = retval.storageType;
+
+      try {
+        blockReader = getBlockReader(targetBlock, offsetIntoBlock,
+            targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
+            storageType, chosenNode);
+        if(connectFailedOnce) {
+          DFSClient.LOG.info("Successfully connected to " + targetAddr +
+                             " for " + targetBlock.getBlock());
+        }
+        return chosenNode;
+      } catch (IOException ex) {
+        if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+              + "encryption key was invalid when connecting to " + targetAddr
+              + " : " + ex);
+          // The encryption key used is invalid.
+          refetchEncryptionKey--;
+          dfsClient.clearDataEncryptionKey();
+        } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
+          refetchToken--;
+          fetchBlockAt(target);
+        } else {
+          connectFailedOnce = true;
+          DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
+            + ", add to deadNodes and continue. " + ex, ex);
+          // Put chosen node into dead list, continue
+          addToDeadNodes(chosenNode);
+        }
+      }
+    }
+  }
+
+  protected BlockReader getBlockReader(LocatedBlock targetBlock,
+      long offsetInBlock, long length, InetSocketAddress targetAddr,
+      StorageType storageType, DatanodeInfo datanode) throws IOException {
+    ExtendedBlock blk = targetBlock.getBlock();
+    Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
+    CachingStrategy curCachingStrategy;
+    boolean shortCircuitForbidden;
+    synchronized (infoLock) {
+      curCachingStrategy = cachingStrategy;
+      shortCircuitForbidden = shortCircuitForbidden();
+    }
+    return new BlockReaderFactory(dfsClient.getConf()).
+        setInetSocketAddress(targetAddr).
+        setRemotePeerFactory(dfsClient).
+        setDatanodeInfo(datanode).
+        setStorageType(storageType).
+        setFileName(src).
+        setBlock(blk).
+        setBlockToken(accessToken).
+        setStartOffset(offsetInBlock).
+        setVerifyChecksum(verifyChecksum).
+        setClientName(dfsClient.clientName).
+        setLength(length).
+        setCachingStrategy(curCachingStrategy).
+        setAllowShortCircuitLocalReads(!shortCircuitForbidden).
+        setClientCacheContext(dfsClient.getClientContext()).
+        setUserGroupInformation(dfsClient.ugi).
+        setConfiguration(dfsClient.getConfiguration()).
+        build();
+  }
+
+  /**
+   * Close it down!
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    if (!closed.compareAndSet(false, true)) {
+      DFSClient.LOG.debug("DFSInputStream has been closed already");
+      return;
+    }
+    dfsClient.checkOpen();
+
+    if ((extendedReadBuffers != null) && (!extendedReadBuffers.isEmpty())) {
+      final StringBuilder builder = new StringBuilder();
+      extendedReadBuffers.visitAll(new IdentityHashStore.Visitor<ByteBuffer, Object>() {
+        private String prefix = "";
+        @Override
+        public void accept(ByteBuffer k, Object v) {
+          builder.append(prefix).append(k);
+          prefix = ", ";
+        }
+      });
+      DFSClient.LOG.warn("closing file " + src + ", but there are still " +
+          "unreleased ByteBuffers allocated by read().  " +
+          "Please release " + builder.toString() + ".");
+    }
+    closeCurrentBlockReaders();
+    super.close();
+  }
+
+  @Override
+  public synchronized int read() throws IOException {
+    if (oneByteBuf == null) {
+      oneByteBuf = new byte[1];
+    }
+    int ret = read( oneByteBuf, 0, 1 );
+    return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
+  }
+
+  /**
+   * Wraps different possible read implementations so that readBuffer can be
+   * strategy-agnostic.
+   */
+  interface ReaderStrategy {
+    public int doRead(BlockReader blockReader, int off, int len)
+        throws ChecksumException, IOException;
+
+    /**
+     * Copy data from the src ByteBuffer into the read buffer.
+     * @param src The src buffer where the data is copied from
+     * @param offset Useful only when the ReadStrategy is based on a byte array.
+     *               Indicate the offset of the byte array for copy.
+     * @param length Useful only when the ReadStrategy is based on a byte array.
+     *               Indicate the length of the data to copy.
+     */
+    public int copyFrom(ByteBuffer src, int offset, int length);
+  }
+
+  protected void updateReadStatistics(ReadStatistics readStatistics,
+        int nRead, BlockReader blockReader) {
+    if (nRead <= 0) return;
+    synchronized(infoLock) {
+      if (blockReader.isShortCircuit()) {
+        readStatistics.addShortCircuitBytes(nRead);
+      } else if (blockReader.isLocal()) {
+        readStatistics.addLocalBytes(nRead);
+      } else {
+        readStatistics.addRemoteBytes(nRead);
+      }
+    }
+  }
+  
+  /**
+   * Used to read bytes into a byte[]
+   */
+  private class ByteArrayStrategy implements ReaderStrategy {
+    final byte[] buf;
+
+    public ByteArrayStrategy(byte[] buf) {
+      this.buf = buf;
+    }
+
+    @Override
+    public int doRead(BlockReader blockReader, int off, int len)
+          throws ChecksumException, IOException {
+      int nRead = blockReader.read(buf, off, len);
+      updateReadStatistics(readStatistics, nRead, blockReader);
+      return nRead;
+    }
+
+    @Override
+    public int copyFrom(ByteBuffer src, int offset, int length) {
+      ByteBuffer writeSlice = src.duplicate();
+      writeSlice.get(buf, offset, length);
+      return length;
+    }
+  }
+
+  /**
+   * Used to read bytes into a user-supplied ByteBuffer
+   */
+  protected class ByteBufferStrategy implements ReaderStrategy {
+    final ByteBuffer buf;
+    ByteBufferStrategy(ByteBuffer buf) {
+      this.buf = buf;
+    }
+
+    @Override
+    public int doRead(BlockReader blockReader, int off, int len)
+        throws ChecksumException, IOException {
+      int oldpos = buf.position();
+      int oldlimit = buf.limit();
+      boolean success = false;
+      try {
+        int ret = blockReader.read(buf);
+        success = true;
+        updateReadStatistics(readStatistics, ret, blockReader);
+        if (ret == 0) {
+          DFSClient.LOG.warn("zero");
+        }
+        return ret;
+      } finally {
+        if (!success) {
+          // Reset to original state so that retries work correctly.
+          buf.position(oldpos);
+          buf.limit(oldlimit);
+        }
+      } 
+    }
+
+    @Override
+    public int copyFrom(ByteBuffer src, int offset, int length) {
+      ByteBuffer writeSlice = src.duplicate();
+      int remaining = Math.min(buf.remaining(), writeSlice.remaining());
+      writeSlice.limit(writeSlice.position() + remaining);
+      buf.put(writeSlice);
+      return remaining;
+    }
+  }
+
+  /* This is a used by regular read() and handles ChecksumExceptions.
+   * name readBuffer() is chosen to imply similarity to readBuffer() in
+   * ChecksumFileSystem
+   */ 
+  private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    IOException ioe;
+    
+    /* we retry current node only once. So this is set to true only here.
+     * Intention is to handle one common case of an error that is not a
+     * failure on datanode or client : when DataNode closes the connection
+     * since client is idle. If there are other cases of "non-errors" then
+     * then a datanode might be retried by setting this to true again.
+     */
+    boolean retryCurrentNode = true;
+
+    while (true) {
+      // retry as many times as seekToNewSource allows.
+      try {
+        return reader.doRead(blockReader, off, len);
+      } catch ( ChecksumException ce ) {
+        DFSClient.LOG.warn("Found Checksum error for "
+            + getCurrentBlock() + " from " + currentNode
+            + " at " + ce.getPos());        
+        ioe = ce;
+        retryCurrentNode = false;
+        // we want to remember which block replicas we have tried
+        addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
+            corruptedBlockMap);
+      } catch ( IOException e ) {
+        if (!retryCurrentNode) {
+          DFSClient.LOG.warn("Exception while reading from "
+              + getCurrentBlock() + " of " + src + " from "
+              + currentNode, e);
+        }
+        ioe = e;
+      }
+      boolean sourceFound = false;
+      if (retryCurrentNode) {
+        /* possibly retry the same node so that transient errors don't
+         * result in application level failures (e.g. Datanode could have
+         * closed the connection because the client is idle for too long).
+         */ 
+        sourceFound = seekToBlockSource(pos);
+      } else {
+        addToDeadNodes(currentNode);
+        sourceFound = seekToNewSource(pos);
+      }
+      if (!sourceFound) {
+        throw ioe;
+      }
+      retryCurrentNode = false;
+    }
+  }
+
+  protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
+    dfsClient.checkOpen();
+    if (closed.get()) {
+      throw new IOException("Stream closed");
+    }
+    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
+      = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+    failures = 0;
+    if (pos < getFileLength()) {
+      int retries = 2;
+      while (retries > 0) {
+        try {
+          // currentNode can be left as null if previous read had a checksum
+          // error on the same block. See HDFS-3067
+          if (pos > blockEnd || currentNode == null) {
+            currentNode = blockSeekTo(pos);
+          }
+          int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
+          synchronized(infoLock) {
+            if (locatedBlocks.isLastBlockComplete()) {
+              realLen = (int) Math.min(realLen,
+                  locatedBlocks.getFileLength() - pos);
+            }
+          }
+          int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
+          
+          if (result >= 0) {
+            pos += result;
+          } else {
+            // got a EOS from reader though we expect more data on it.
+            throw new IOException("Unexpected EOS from the reader");
+          }
+          if (dfsClient.stats != null) {
+            dfsClient.stats.incrementBytesRead(result);
+          }
+          return result;
+        } catch (ChecksumException ce) {
+          throw ce;            
+        } catch (IOException e) {
+          if (retries == 1) {
+            DFSClient.LOG.warn("DFS Read", e);
+          }
+          blockEnd = -1;
+          if (currentNode != null) { addToDeadNodes(currentNode); }
+          if (--retries == 0) {
+            throw e;
+          }
+        } finally {
+          // Check if need to report block replicas corruption either read
+          // was successful or ChecksumException occured.
+          reportCheckSumFailure(corruptedBlockMap, 
+              currentLocatedBlock.getLocations().length);
+        }
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Read the entire buffer.
+   */
+  @Override
+  public synchronized int read(final byte buf[], int off, int len) throws IOException {
+    ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
+    TraceScope scope =
+        dfsClient.getPathTraceScope("DFSInputStream#byteArrayRead", src);
+    try {
+      return readWithStrategy(byteArrayReader, off, len);
+    } finally {
+      scope.close();
+    }
+  }
+
+  @Override
+  public synchronized int read(final ByteBuffer buf) throws IOException {
+    ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
+    TraceScope scope =
+        dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
+    try {
+      return readWithStrategy(byteBufferReader, 0, buf.remaining());
+    } finally {
+      scope.close();
+    }
+  }
+
+
+  /**
+   * Add corrupted block replica into map.
+   */
+  protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+    Set<DatanodeInfo> dnSet = null;
+    if((corruptedBlockMap.containsKey(blk))) {
+      dnSet = corruptedBlockMap.get(blk);
+    }else {
+      dnSet = new HashSet<DatanodeInfo>();
+    }
+    if (!dnSet.contains(node)) {
+      dnSet.add(node);
+      corruptedBlockMap.put(blk, dnSet);
+    }
+  }
+
+  private DNAddrPair chooseDataNode(LocatedBlock block,
+      Collection<DatanodeInfo> ignoredNodes) throws IOException {
+    while (true) {
+      DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
+      if (result != null) {
+        return result;
+      } else {
+        String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
+          deadNodes, ignoredNodes);
+        String blockInfo = block.getBlock() + " file=" + src;
+        if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
+          String description = "Could not obtain block: " + blockInfo;
+          DFSClient.LOG.warn(description + errMsg
+              + ". Throwing a BlockMissingException");
+          throw new BlockMissingException(src, description,
+              block.getStartOffset());
+        }
+
+        DatanodeInfo[] nodes = block.getLocations();
+        if (nodes == null || nodes.length == 0) {
+          DFSClient.LOG.info("No node available for " + blockInfo);
+        }
+        DFSClient.LOG.info("Could not obtain " + block.getBlock()
+            + " from any node: " + errMsg
+            + ". Will get new block locations from namenode and retry...");
+        try {
+          // Introducing a random factor to the wait time before another retry.
+          // The wait time is dependent on # of failures and a random factor.
+          // At the first time of getting a BlockMissingException, the wait time
+          // is a random number between 0..3000 ms. If the first retry
+          // still fails, we will wait 3000 ms grace period before the 2nd retry.
+          // Also at the second retry, the waiting window is expanded to 6000 ms
+          // alleviating the request rate from the server. Similarly the 3rd retry
+          // will wait 6000ms grace period before retry and the waiting window is
+          // expanded to 9000ms. 
+          final int timeWindow = dfsClient.getConf().getTimeWindow();
+          double waitTime = timeWindow * failures +       // grace period for the last round of attempt
+              // expanding time window for each failure
+              timeWindow * (failures + 1) *
+              ThreadLocalRandom.current().nextDouble();
+          DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
+          Thread.sleep((long)waitTime);
+        } catch (InterruptedException iex) {
+        }
+        deadNodes.clear(); //2nd option is to remove only nodes[blockId]
+        openInfo(true);
+        block = refreshLocatedBlock(block);
+        failures++;
+      }
+    }
+  }
+
+  /**
+   * Get the best node from which to stream the data.
+   * @param block LocatedBlock, containing nodes in priority order.
+   * @param ignoredNodes Do not choose nodes in this array (may be null)
+   * @return The DNAddrPair of the best node. Null if no node can be chosen.
+   */
+  protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
+      Collection<DatanodeInfo> ignoredNodes) {
+    DatanodeInfo[] nodes = block.getLocations();
+    StorageType[] storageTypes = block.getStorageTypes();
+    DatanodeInfo chosenNode = null;
+    StorageType storageType = null;
+    if (nodes != null) {
+      for (int i = 0; i < nodes.length; i++) {
+        if (!deadNodes.containsKey(nodes[i])
+            && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
+          chosenNode = nodes[i];
+          // Storage types are ordered to correspond with nodes, so use the same
+          // index to get storage type.
+          if (storageTypes != null && i < storageTypes.length) {
+            storageType = storageTypes[i];
+          }
+          break;
+        }
+      }
+    }
+    if (chosenNode == null) {
+      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
+          " after checking nodes = " + Arrays.toString(nodes) +
+          ", ignoredNodes = " + ignoredNodes);
+      return null;
+    }
+    final String dnAddr =
+        chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+    }
+    InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
+    return new DNAddrPair(chosenNode, targetAddr, storageType);
+  }
+
+  private static String getBestNodeDNAddrPairErrorString(
+      DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
+      DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
+    StringBuilder errMsgr = new StringBuilder(
+        " No live nodes contain current block ");
+    errMsgr.append("Block locations:");
+    for (DatanodeInfo datanode : nodes) {
+      errMsgr.append(" ");
+      errMsgr.append(datanode.toString());
+    }
+    errMsgr.append(" Dead nodes: ");
+    for (DatanodeInfo datanode : deadNodes.keySet()) {
+      errMsgr.append(" ");
+      errMsgr.append(datanode.toString());
+    }
+    if (ignoredNodes != null) {
+      errMsgr.append(" Ignored nodes: ");
+      for (DatanodeInfo datanode : ignoredNodes) {
+        errMsgr.append(" ");
+        errMsgr.append(datanode.toString());
+      }
+    }
+    return errMsgr.toString();
+  }
+
+  protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
+      byte[] buf, int offset,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    block = refreshLocatedBlock(block);
+    while (true) {
+      DNAddrPair addressPair = chooseDataNode(block, null);
+      try {
+        actualGetFromOneDataNode(addressPair, block, start, end,
+            buf, offset, corruptedBlockMap);
+        return;
+      } catch (IOException e) {
+        // Ignore. Already processed inside the function.
+        // Loop through to try the next node.
+      }
+    }
+  }
+
+  private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
+      final LocatedBlock block, final long start, final long end,
+      final ByteBuffer bb,
+      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+      final int hedgedReadId) {
+    final Span parentSpan = Trace.currentSpan();
+    return new Callable<ByteBuffer>() {
+      @Override
+      public ByteBuffer call() throws Exception {
+        byte[] buf = bb.array();
+        int offset = bb.position();
+        TraceScope scope =
+            Trace.startSpan("hedgedRead" + hedgedReadId, parentSpan);
+        try {
+          actualGetFromOneDataNode(datanode, block, start, end, buf,
+              offset, corruptedBlockMap);
+          return bb;
+        } finally {
+          scope.close();
+        }
+      }
+    };
+  }
+
+  /**
+   * Used when reading contiguous blocks
+   */
+  private void actualGetFromOneDataNode(final DNAddrPair datanode,
+      LocatedBlock block, final long start, final long end, byte[] buf,
+      int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    final int length = (int) (end - start + 1);
+    actualGetFromOneDataNode(datanode, block, start, end, buf,
+        new int[]{offset}, new int[]{length}, corruptedBlockMap);
+  }
+
+  /**
+   * Read data from one DataNode.
+   * @param datanode the datanode from which to read data
+   * @param block the located block containing the requested data
+   * @param startInBlk the startInBlk offset of the block
+   * @param endInBlk the endInBlk offset of the block
+   * @param buf the given byte array into which the data is read
+   * @param offsets the data may be read into multiple segments of the buf
+   *                (when reading a striped block). this array indicates the
+   *                offset of each buf segment.
+   * @param lengths the length of each buf segment
+   * @param corruptedBlockMap map recording list of datanodes with corrupted
+   *                          block replica
+   */
+  void actualGetFromOneDataNode(final DNAddrPair datanode,
+      LocatedBlock block, final long startInBlk, final long endInBlk,
+      byte[] buf, int[] offsets, int[] lengths,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    DFSClientFaultInjector.get().startFetchFromDatanode();
+    int refetchToken = 1; // only need to get a new access token once
+    int refetchEncryptionKey = 1; // only need to get a new encryption key once
+    final int len = (int) (endInBlk - startInBlk + 1);
+    checkReadPortions(offsets, lengths, len);
+
+    while (true) {
+      // cached block locations may have been updated by chooseDataNode()
+      // or fetchBlockAt(). Always get the latest list of locations at the
+      // start of the loop.
+      block = refreshLocatedBlock(block);
+      BlockReader reader = null;
+      try {
+        DFSClientFaultInjector.get().fetchFromDatanodeException();
+        reader = getBlockReader(block, startInBlk, len, datanode.addr,
+            datanode.storageType, datanode.info);
+        for (int i = 0; i < offsets.length; i++) {
+          int nread = reader.readAll(buf, offsets[i], lengths[i]);
+          updateReadStatistics(readStatistics, nread, reader);
+          if (nread != lengths[i]) {
+            throw new IOException("truncated return from reader.read(): " +
+                "excpected " + lengths[i] + ", got " + nread);
+          }
+        }
+        DFSClientFaultInjector.get().readFromDatanodeDelay();
+        return;
+      } catch (ChecksumException e) {
+        String msg = "fetchBlockByteRange(). Got a checksum exception for "
+            + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
+            + datanode.info;
+        DFSClient.LOG.warn(msg);
+        // we want to remember what we have tried
+        addIntoCorruptedBlockMap(block.getBlock(), datanode.info,
+            corruptedBlockMap);
+        addToDeadNodes(datanode.info);
+        throw new IOException(msg);
+      } catch (IOException e) {
+        if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+          DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
+              + "encryption key was invalid when connecting to " + datanode.addr
+              + " : " + e);
+          // The encryption key used is invalid.
+          refetchEncryptionKey--;
+          dfsClient.clearDataEncryptionKey();
+        } else if (refetchToken > 0 && tokenRefetchNeeded(e, datanode.addr)) {
+          refetchToken--;
+          try {
+            fetchBlockAt(block.getStartOffset());
+          } catch (IOException fbae) {
+            // ignore IOE, since we can retry it later in a loop
+          }
+        } else {
+          String msg = "Failed to connect to " + datanode.addr + " for file "
+              + src + " for block " + block.getBlock() + ":" + e;
+          DFSClient.LOG.warn("Connection failure: " + msg, e);
+          addToDeadNodes(datanode.info);
+          throw new IOException(msg);
+        }
+      } finally {
+        if (reader != null) {
+          reader.close();
+        }
+      }
+    }
+  }
+
+  /**
+   * Refresh cached block locations.
+   * @param block The currently cached block locations
+   * @return Refreshed block locations
+   * @throws IOException
+   */
+  protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
+      throws IOException {
+    return getBlockAt(block.getStartOffset());
+  }
+
+  /**
+   * This method verifies that the read portions are valid and do not overlap
+   * with each other.
+   */
+  private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
+    Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
+    int sum = 0;
+    for (int i = 0; i < lengths.length; i++) {
+      if (i > 0) {
+        int gap = offsets[i] - offsets[i - 1];
+        // make sure read portions do not overlap with each other
+        Preconditions.checkArgument(gap >= lengths[i - 1]);
+      }
+      sum += lengths[i];
+    }
+    Preconditions.checkArgument(sum == totalLen);
+  }
+
+  /**
+   * Like {@link #fetchBlockByteRange}except we start up a second, parallel,
+   * 'hedged' read if the first read is taking longer than configured amount of
+   * time. We then wait on which ever read returns first.
+   */
+  private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
+      long end, byte[] buf, int offset,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    final DfsClientConf conf = dfsClient.getConf();
+    ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
+    CompletionService<ByteBuffer> hedgedService =
+        new ExecutorCompletionService<ByteBuffer>(
+        dfsClient.getHedgedReadsThreadPool());
+    ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
+    ByteBuffer bb = null;
+    int len = (int) (end - start + 1);
+    int hedgedReadId = 0;
+    block = refreshLocatedBlock(block);
+    while (true) {
+      // see HDFS-6591, this metric is used to verify/catch unnecessary loops
+      hedgedReadOpsLoopNumForTesting++;
+      DNAddrPair chosenNode = null;
+      // there is no request already executing.
+      if (futures.isEmpty()) {
+        // chooseDataNode is a commitment. If no node, we go to
+        // the NN to reget block locations. Only go here on first read.
+        chosenNode = chooseDataNode(block, ignored);
+        bb = ByteBuffer.wrap(buf, offset, len);
+        Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
+            chosenNode, block, start, end, bb,
+            corruptedBlockMap, hedgedReadId++);
+        Future<ByteBuffer> firstRequest = hedgedService
+            .submit(getFromDataNodeCallable);
+        futures.add(firstRequest);
+        try {
+          Future<ByteBuffer> future = hedgedService.poll(
+              conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
+          if (future != null) {
+            future.get();
+            return;
+          }
+          if (DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
+                + "ms to read from " + chosenNode.info
+                + "; spawning hedged read");
+          }
+          // Ignore this node on next go around.
+          ignored.add(chosenNode.info);
+          dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+          continue; // no need to refresh block locations
+        } catch (InterruptedException e) {
+          // Ignore
+        } catch (ExecutionException e) {
+          // Ignore already logged in the call.
+        }
+      } else {
+        // We are starting up a 'hedged' read. We have a read already
+        // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
+        // If no nodes to do hedged reads against, pass.
+        try {
+          chosenNode = getBestNodeDNAddrPair(block, ignored);
+          if (chosenNode == null) {
+            chosenNode = chooseDataNode(block, ignored);
+          }
+          bb = ByteBuffer.allocate(len);
+          Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
+              chosenNode, block, start, end, bb,
+              corruptedBlockMap, hedgedReadId++);
+          Future<ByteBuffer> oneMoreRequest = hedgedService
+              .submit(getFromDataNodeCallable);
+          futures.add(oneMoreRequest);
+        } catch (IOException ioe) {
+          if (DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Failed getting node for hedged read: "
+                + ioe.getMessage());
+          }
+        }
+        // if not succeeded. Submit callables for each datanode in a loop, wait
+        // for a fixed interval and get the result from the fastest one.
+        try {
+          ByteBuffer result = getFirstToComplete(hedgedService, futures);
+          // cancel the rest.
+          cancelAll(futures);
+          if (result.array() != buf) { // compare the array pointers
+            dfsClient.getHedgedReadMetrics().incHedgedReadWins();
+            System.arraycopy(result.array(), result.position(), buf, offset,
+                len);
+          } else {
+            dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+          }
+          return;
+        } catch (InterruptedException ie) {
+          // Ignore and retry
+        }
+        // We got here if exception. Ignore this node on next go around IFF
+        // we found a chosenNode to hedge read against.
+        if (chosenNode != null && chosenNode.info != null) {
+          ignored.add(chosenNode.info);
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public long getHedgedReadOpsLoopNumForTesting() {
+    return hedgedReadOpsLoopNumForTesting;
+  }
+
+  private ByteBuffer getFirstToComplete(
+      CompletionService<ByteBuffer> hedgedService,
+      ArrayList<Future<ByteBuffer>> futures) throws InterruptedException {
+    if (futures.isEmpty()) {
+      throw new InterruptedException("let's retry");
+    }
+    Future<ByteBuffer> future = null;
+    try {
+      future = hedgedService.take();
+      ByteBuffer bb = future.get();
+      futures.remove(future);
+      return bb;
+    } catch (ExecutionException e) {
+      // already logged in the Callable
+      futures.remove(future);
+    } catch (CancellationException ce) {
+      // already logged in the Callable
+      futures.remove(future);
+    }
+
+    throw new InterruptedException("let's retry");
+  }
+
+  private void cancelAll(List<Future<ByteBuffer>> futures) {
+    for (Future<ByteBuffer> future : futures) {
+      // Unfortunately, hdfs reads do not take kindly to interruption.
+      // Threads return a variety of interrupted-type exceptions but
+      // also complaints about invalid pbs -- likely because read
+      // is interrupted before gets whole pb.  Also verbose WARN
+      // logging.  So, for now, do not interrupt running read.
+      future.cancel(false);
+    }
+  }
+
+  /**
+   * Should the block access token be refetched on an exception
+   * 
+   * @param ex Exception received
+   * @param targetAddr Target datanode address from where exception was received
+   * @return true if block access token has expired or invalid and it should be
+   *         refetched
+   */
+  protected static boolean tokenRefetchNeeded(IOException ex,
+      InetSocketAddress targetAddr) {
+    /*
+     * Get a new access token and retry. Retry is needed in 2 cases. 1)
+     * When both NN and DN re-started while DFSClient holding a cached
+     * access token. 2) In the case that NN fails to update its
+     * access key at pre-set interval (by a wide margin) and
+     * subsequently restarts. In this case, DN re-registers itself with
+     * NN and receives a new access key, but DN will delete the old
+     * access key from its memory since it's considered expired based on
+     * the estimated expiration date.
+     */
+    if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
+      DFSClient.LOG.info("Access token was invalid when connecting to "
+          + targetAddr + " : " + ex);
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Read bytes starting from the specified position.
+   * 
+   * @param position start read from this position
+   * @param buffer read buffer
+   * @param offset offset into buffer
+   * @param length number of bytes to read
+   * 
+   * @return actual number of bytes read
+   */
+  @Override
+  public int read(long position, byte[] buffer, int offset, int length)
+      throws IOException {
+    TraceScope scope =
+        dfsClient.getPathTraceScope("DFSInputStream#byteArrayPread", src);
+    try {
+      return pread(position, buffer, offset, length);
+    } finally {
+      scope.close();
+    }
+  }
+
+  private int pread(long position, byte[] buffer, int offset, int length)
+      throws IOException {
+    // sanity checks
+    dfsClient.checkOpen();
+    if (closed.get()) {
+      throw new IOException("Stream closed");
+    }
+    failures = 0;
+    long filelen = getFileLength();
+    if ((position < 0) || (position >= filelen)) {
+      return -1;
+    }
+    int realLen = length;
+    if ((position + length) > filelen) {
+      realLen = (int)(filelen - position);
+    }
+    
+    // determine the block and byte range within the block
+    // corresponding to position and realLen
+    List<LocatedBlock> blockRange = getBlockRange(position, realLen);
+    int remaining = realLen;
+    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
+      = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+    for (LocatedBlock blk : blockRange) {
+      long targetStart = position - blk.getStartOffset();
+      long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
+      try {
+        if (dfsClient.isHedgedReadsEnabled()) {
+          hedgedFetchBlockByteRange(blk, targetStart,
+              targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
+        } else {
+          fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
+              buffer, offset, corruptedBlockMap);
+        }
+      } finally {
+        // Check and report if any block replicas are corrupted.
+        // BlockMissingException may be caught if all block replicas are
+        // corrupted.
+        reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
+      }
+
+      remaining -= bytesToRead;
+      position += bytesToRead;
+      offset += bytesToRead;
+    }
+    assert remaining == 0 : "Wrong number of bytes read.";
+    if (dfsClient.stats != null) {
+      dfsClient.stats.incrementBytesRead(realLen);
+    }
+    return realLen;
+  }
+  
+  /**
+   * DFSInputStream reports checksum failure.
+   * Case I : client has tried multiple data nodes and at least one of the
+   * attempts has succeeded. We report the other failures as corrupted block to
+   * namenode. 
+   * Case II: client has tried out all data nodes, but all failed. We
+   * only report if the total number of replica is 1. We do not
+   * report otherwise since this maybe due to the client is a handicapped client
+   * (who can not read).
+   * @param corruptedBlockMap map of corrupted blocks
+   * @param dataNodeCount number of data nodes who contains the block replicas
+   */
+  protected void reportCheckSumFailure(
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap, 
+      int dataNodeCount) {
+    if (corruptedBlockMap.isEmpty()) {
+      return;
+    }
+    Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
+        .entrySet().iterator();
+    Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
+    ExtendedBlock blk = entry.getKey();
+    Set<DatanodeInfo> dnSet = entry.getValue();
+    if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
+        || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
+      DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
+      int i = 0;
+      for (DatanodeInfo dn:dnSet) {
+        locs[i++] = dn;
+      }
+      LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
+      dfsClient.reportChecksumFailure(src, lblocks);
+    }
+    corruptedBlockMap.clear();
+  }
+
+  @Override
+  public long skip(long n) throws IOException {
+    if ( n > 0 ) {
+      long curPos = getPos();
+      long fileLen = getFileLength();
+      if( n+curPos > fileLen ) {
+        n = fileLen - curPos;
+      }
+      seek(curPos+n);
+      return n;
+    }
+    return n < 0 ? -1 : 0;
+  }
+
+  /**
+   * Seek to a new arbitrary location
+   */
+  @Override
+  public synchronized void seek(long targetPos) throws IOException {
+    if (targetPos > getFileLength()) {
+      throw new EOFException("Cannot seek after EOF");
+    }
+    if (targetPos < 0) {
+      throw new EOFException("Cannot seek to negative offset");
+    }
+    if (closed.get()) {
+      throw new IOException("Stream is closed!");
+    }
+    boolean done = false;
+    if (pos <= targetPos && targetPos <= blockEnd) {
+      //
+      // If this seek is to a positive position in the current
+      // block, and this piece of data might already be lying in
+      // the TCP buffer, then just eat up the intervening data.
+      //
+      int diff = (int)(targetPos - pos);
+      if (diff <= blockReader.available()) {
+        try {
+          pos += blockReader.skip(diff);
+          if (pos == targetPos) {
+            done = true;
+          } else {
+            // The range was already checked. If the block reader returns
+            // something unexpected instead of throwing an exception, it is
+            // most likely a bug. 
+            String errMsg = "BlockReader failed to seek to " + 
+                targetPos + ". Instead, it seeked to " + pos + ".";
+            DFSClient.LOG.warn(errMsg);
+            throw new IOException(errMsg);
+          }
+        } catch (IOException e) {//make following read to retry
+          if(DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Exception while seek to " + targetPos
+                + " from " + getCurrentBlock() + " of " + src + " from "
+                + currentNode, e);
+          }
+        }
+      }
+    }
+    if (!done) {
+      pos = targetPos;
+      blockEnd = -1;
+    }
+  }
+
+  /**
+   * Same as {@link #seekToNewSource(long)} except that it does not exclude
+   * the current datanode and might connect to the same node.
+   */
+  private boolean seekToBlockSource(long targetPos)
+                                                 throws IOException {
+    currentNode = blockSeekTo(targetPos);
+    return true;
+  }
+  
+  /**
+   * Seek to given position on a node other than the current node.  If
+   * a node other than the current node is found, then returns true. 
+   * If another node could not be found, then returns false.
+   */
+  @Override
+  public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+    if (currentNode == null) {
+      return seekToBlockSource(targetPos);
+    }
+    boolean markedDead = deadNodes.containsKey(currentNode);
+    addToDeadNodes(currentNode);
+    DatanodeInfo oldNode = currentNode;
+    DatanodeInfo newNode = blockSeekTo(targetPos);
+    if (!markedDead) {
+      /* remove it from deadNodes. blockSeekTo could have cleared 
+       * deadNodes and added currentNode again. Thats ok. */
+      deadNodes.remove(oldNode);
+    }
+    if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
+      currentNode = newNode;
+      return true;
+    } else {
+      return false;
+    }
+  }
+      
+  /**
+   */
+  @Override
+  public synchronized long getPos() {
+    return pos;
+  }
+
+  /** Return the size of the remaining available bytes
+   * if the size is less than or equal to {@link Integer#MAX_VALUE},
+   * otherwise, return {@link Integer#MAX_VALUE}.
+   */
+  @Override
+  public synchronized int available() throws IOException {
+    if (closed.get()) {
+      throw new IOException("Stream closed");
+    }
+
+    final long remaining = getFileLength() - pos;
+    return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
+  }
+
+  /**
+   * We definitely don't support marks
+   */
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+  @Override
+  public void mark(int readLimit) {
+  }
+  @Override
+  public void reset() throws IOException {
+    throw new IOException("Mark/reset not supported");
+  }
+
+  /** Utility class to encapsulate data node info and its address. */
+  static final class DNAddrPair {
+    final DatanodeInfo info;
+    final InetSocketAddress addr;
+    final StorageType storageType;
+
+    DNAddrPair(DatanodeInfo info, InetSocketAddress addr,
+        StorageType storageType) {
+      this.info = info;
+      this.addr = addr;
+      this.storageType = storageType;
+    }
+  }
+
+  /**
+   * Get statistics about the reads which this DFSInputStream has done.
+   */
+  public ReadStatistics getReadStatistics() {
+    synchronized(infoLock) {
+      return new ReadStatistics(readStatistics);
+    }
+  }
+
+  /**
+   * Clear statistics about the reads which this DFSInputStream has done.
+   */
+  public void clearReadStatistics() {
+    synchronized(infoLock) {
+      readStatistics.clear();
+    }
+  }
+
+  public FileEncryptionInfo getFileEncryptionInfo() {
+    synchronized(infoLock) {
+      return fileEncryptionInfo;
+    }
+  }
+
+  protected void closeCurrentBlockReaders() {
+    if (blockReader == null) return;
+    // Close the current block reader so that the new caching settings can 
+    // take effect immediately.
+    try {
+      blockReader.close();
+    } catch (IOException e) {
+      DFSClient.LOG.error("error closing blockReader", e);
+    }
+    blockReader = null;
+    blockEnd = -1;
+  }
+
+  @Override
+  public synchronized void setReadahead(Long readahead)
+      throws IOException {
+    synchronized (infoLock) {
+      this.cachingStrategy =
+          new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
+    }
+    closeCurrentBlockReaders();
+  }
+
+  @Override
+  public synchronized void setDropBehind(Boolean dropBehind)
+      throws IOException {
+    synchronized (infoLock) {
+      this.cachingStrategy =
+          new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
+    }
+    closeCurrentBlockReaders();
+  }
+
+  /**
+   * The immutable empty buffer we return when we reach EOF when doing a
+   * zero-copy read.
+   */
+  private static final ByteBuffer EMPTY_BUFFER =
+    ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
+
+  @Override
+  public synchronized ByteBuffer read(ByteBufferPool bufferPool,
+      int maxLength, EnumSet<ReadOption> opts) 
+          throws IOException, UnsupportedOperationException {
+    if (maxLength == 0) {
+      return EMPTY_BUFFER;
+    } else if (maxLength < 0) {
+      throw new IllegalArgumentException("can't read a negative " +
+          "number of bytes.");
+    }
+    if ((blockReader == null) || (blockEnd == -1)) {
+      if (pos >= getFileLength()) {
+        return null;
+      }
+      /*
+       * If we don't have a blockReader, or the one we have has no more bytes
+       * left to read, we call seekToBlockSource to get a new blockReader and
+       * recalculate blockEnd.  Note that we assume we're not at EOF here
+       * (we check this above).
+       */
+      if ((!seekToBlockSource(pos)) || (blockReader == null)) {
+        throw new IOException("failed to allocate new BlockReader " +
+            "at position " + pos);
+      }
+    }
+    ByteBuffer buffer = null;
+    if (dfsClient.getConf().getShortCircuitConf().isShortCircuitMmapEnabled()) {
+      buffer = tryReadZeroCopy(maxLength, opts);
+    }
+    if (buffer != null) {
+      return buffer;
+    }
+    buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
+    if (buffer != null) {
+      getExtendedReadBuffers().put(buffer, bufferPool);
+    }
+    return buffer;
+  }
+
+  private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
+      EnumSet<ReadOption> opts) throws IOException {
+    // Copy 'pos' and 'blockEnd' to local variables to make it easier for the
+    // JVM to optimize this function.
+    final long curPos = pos;
+    final long curEnd = blockEnd;
+    final long blockStartInFile = currentLocatedBlock.getStartOffset();
+    final long blockPos = curPos - blockStartInFile;
+
+    // Shorten this read if the end of the block is nearby.
+    long length63;
+    if ((curPos + maxLength) <= (curEnd + 1)) {
+      length63 = maxLength;
+    } else {
+      length63 = 1 + curEnd - curPos;
+      if (length63 <= 0) {
+        if (DFSClient.LOG.isDebugEnabled()) {
+          DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
+            curPos + " of " + src + "; " + length63 + " bytes left in block.  " +
+            "blockPos=" + blockPos + "; curPos=" + curPos +
+            "; curEnd=" + curEnd);
+        }
+        return null;
+      }
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("Reducing read length from " + maxLength +
+            " to " + length63 + " to avoid going more than one byte " +
+            "past the end of the block.  blockPos=" + blockPos +
+            "; curPos=" + curPos + "; curEnd=" + curEnd);
+      }
+    }
+    // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
+    int length;
+    if (blockPos + length63 <= Integer.MAX_VALUE) {
+      length = (int)length63;
+    } else {
+      long length31 = Integer.MAX_VALUE - blockPos;
+      if (length31 <= 0) {
+        // Java ByteBuffers can't be longer than 2 GB, because they use
+        // 4-byte signed integers to represent capacity, etc.
+        // So we can't mmap the parts of the block higher than the 2 GB offset.
+        // FIXME: we could work around this with multiple memory maps.
+        // See HDFS-5101.
+        if (DFSClient.LOG.isDebugEnabled()) {
+          DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
+            curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
+            "exceeded.  blockPos=" + blockPos + ", curEnd=" + curEnd);
+        }
+        return null;
+      }
+      length = (int)length31;
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("Reducing read length from " + maxLength +
+            " to " + length + " to avoid 31-bit limit.  " +
+            "blockPos=" + blockPos + "; curPos=" + curPos +
+            "; curEnd=" + curEnd);
+      }
+    }
+    final ClientMmap clientMmap = blockReader.getClientMmap(opts);
+    if (clientMmap == null) {
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
+          curPos + " of " + src + "; BlockReader#getClientMmap returned " +
+          "null.");
+      }
+      return null;
+    }
+    boolean success = false;
+    ByteBuffer buffer;
+    try {
+      seek(curPos + length);
+      buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
+      buffer.position((int)blockPos);
+      buffer.limit((int)(blockPos + length));
+      getExtendedReadBuffers().put(buffer, clientMmap);
+      synchronized (infoLock) {
+        readStatistics.addZeroCopyBytes(length);
+      }
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("readZeroCopy read " + length + 
+            " bytes from offset " + curPos + " via the zero-copy read " +
+            "path.  blockEnd = " + blockEnd);
+      }
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeQuietly(clientMmap);
+      }
+    }
+    return buffer;
+  }
+
+  @Override
+  public synchronized void releaseBuffer(ByteBuffer buffer) {
+    if (buffer == EMPTY_BUFFER) return;
+    Object val = getExtendedReadBuffers().remove(buffer);
+    if (val == null) {
+      throw new IllegalArgumentException("tried to release a buffer " +
+          "that was not created by this stream, " + buffer);
+    }
+    if (val instanceof ClientMmap) {
+      IOUtils.closeQuietly((ClientMmap)val);
+    } else if (val instanceof ByteBufferPool) {
+      ((ByteBufferPool)val).putBuffer(buffer);
+    }
+  }
+
+  @Override
+  public synchronized void unbuffer() {
+    closeCurrentBlockReaders();
+  }
+}


[05/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
deleted file mode 100644
index 3b09953..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ /dev/null
@@ -1,3200 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.Proxy;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.security.GeneralSecurityException;
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.net.SocketFactory;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherSuite;
-import org.apache.hadoop.crypto.CryptoCodec;
-import org.apache.hadoop.crypto.CryptoInputStream;
-import org.apache.hadoop.crypto.CryptoOutputStream;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.BlockStorageLocation;
-import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.FsStatus;
-import org.apache.hadoop.fs.HdfsBlockLocation;
-import org.apache.hadoop.fs.InvalidPathException;
-import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
-import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
-import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Options.ChecksumOpt;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.VolumeId;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.AclException;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
-import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
-import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
-import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
-import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.RpcInvocationHandler;
-import org.apache.hadoop.net.DNS;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenRenewer;
-import org.apache.hadoop.tracing.SpanReceiverHost;
-import org.apache.hadoop.tracing.TraceUtils;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.DataChecksum.Type;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.SamplerBuilder;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.net.InetAddresses;
-
-/********************************************************
- * DFSClient can connect to a Hadoop Filesystem and 
- * perform basic file tasks.  It uses the ClientProtocol
- * to communicate with a NameNode daemon, and connects 
- * directly to DataNodes to read/write block data.
- *
- * Hadoop DFS users should obtain an instance of 
- * DistributedFileSystem, which uses DFSClient to handle
- * filesystem tasks.
- *
- ********************************************************/
-@InterfaceAudience.Private
-public class DFSClient implements java.io.Closeable, RemotePeerFactory,
-    DataEncryptionKeyFactory {
-  public static final Log LOG = LogFactory.getLog(DFSClient.class);
-  public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
-
-  private final Configuration conf;
-  private final DfsClientConf dfsClientConf;
-  final ClientProtocol namenode;
-  /* The service used for delegation tokens */
-  private Text dtService;
-
-  final UserGroupInformation ugi;
-  volatile boolean clientRunning = true;
-  volatile long lastLeaseRenewal;
-  private volatile FsServerDefaults serverDefaults;
-  private volatile long serverDefaultsLastUpdate;
-  final String clientName;
-  final SocketFactory socketFactory;
-  final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
-  final FileSystem.Statistics stats;
-  private final String authority;
-  private final Random r = new Random();
-  private SocketAddress[] localInterfaceAddrs;
-  private DataEncryptionKey encryptionKey;
-  final SaslDataTransferClient saslClient;
-  private final CachingStrategy defaultReadCachingStrategy;
-  private final CachingStrategy defaultWriteCachingStrategy;
-  private final ClientContext clientContext;
-
-  private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
-      new DFSHedgedReadMetrics();
-  private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
-  private final Sampler<?> traceSampler;
-  private final int smallBufferSize;
-
-  public DfsClientConf getConf() {
-    return dfsClientConf;
-  }
-
-  Configuration getConfiguration() {
-    return conf;
-  }
-
-  /**
-   * A map from file names to {@link DFSOutputStream} objects
-   * that are currently being written by this client.
-   * Note that a file can only be written by a single client.
-   */
-  private final Map<Long, DFSOutputStream> filesBeingWritten
-      = new HashMap<Long, DFSOutputStream>();
-
-  /**
-   * Same as this(NameNode.getNNAddress(conf), conf);
-   * @see #DFSClient(InetSocketAddress, Configuration)
-   * @deprecated Deprecated at 0.21
-   */
-  @Deprecated
-  public DFSClient(Configuration conf) throws IOException {
-    this(DFSUtilClient.getNNAddress(conf), conf);
-  }
-  
-  public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
-    this(DFSUtilClient.getNNUri(address), conf);
-  }
-
-  /**
-   * Same as this(nameNodeUri, conf, null);
-   * @see #DFSClient(URI, Configuration, FileSystem.Statistics)
-   */
-  public DFSClient(URI nameNodeUri, Configuration conf
-      ) throws IOException {
-    this(nameNodeUri, conf, null);
-  }
-
-  /**
-   * Same as this(nameNodeUri, null, conf, stats);
-   * @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics) 
-   */
-  public DFSClient(URI nameNodeUri, Configuration conf,
-                   FileSystem.Statistics stats)
-    throws IOException {
-    this(nameNodeUri, null, conf, stats);
-  }
-  
-  /** 
-   * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
-   * If HA is enabled and a positive value is set for
-   * {@link HdfsClientConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY}
-   * in the configuration, the DFSClient will use
-   * {@link LossyRetryInvocationHandler} as its RetryInvocationHandler.
-   * Otherwise one of nameNodeUri or rpcNamenode must be null.
-   */
-  @VisibleForTesting
-  public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
-      Configuration conf, FileSystem.Statistics stats)
-    throws IOException {
-    SpanReceiverHost.get(conf, HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
-    traceSampler = new SamplerBuilder(TraceUtils.
-        wrapHadoopConf(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf))
-        .build();
-    // Copy only the required DFSClient configuration
-    this.dfsClientConf = new DfsClientConf(conf);
-    this.conf = conf;
-    this.stats = stats;
-    this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
-    this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
-    this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
-
-    this.ugi = UserGroupInformation.getCurrentUser();
-    
-    this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
-    this.clientName = "DFSClient_" + dfsClientConf.getTaskId() + "_" + 
-        ThreadLocalRandom.current().nextInt()  + "_" +
-        Thread.currentThread().getId();
-    int numResponseToDrop = conf.getInt(
-        HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
-        HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
-    ProxyAndInfo<ClientProtocol> proxyInfo = null;
-    AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
-    if (numResponseToDrop > 0) {
-      // This case is used for testing.
-      LOG.warn(HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
-          + " is set to " + numResponseToDrop
-          + ", this hacked client will proactively drop responses");
-      proxyInfo = NameNodeProxiesClient.createProxyWithLossyRetryHandler(conf,
-          nameNodeUri, ClientProtocol.class, numResponseToDrop,
-          nnFallbackToSimpleAuth);
-    }
-    
-    if (proxyInfo != null) {
-      this.dtService = proxyInfo.getDelegationTokenService();
-      this.namenode = proxyInfo.getProxy();
-    } else if (rpcNamenode != null) {
-      // This case is used for testing.
-      Preconditions.checkArgument(nameNodeUri == null);
-      this.namenode = rpcNamenode;
-      dtService = null;
-    } else {
-      Preconditions.checkArgument(nameNodeUri != null,
-          "null URI");
-      proxyInfo = NameNodeProxiesClient.createProxyWithClientProtocol(conf,
-          nameNodeUri, nnFallbackToSimpleAuth);
-      this.dtService = proxyInfo.getDelegationTokenService();
-      this.namenode = proxyInfo.getProxy();
-    }
-
-    String localInterfaces[] =
-      conf.getTrimmedStrings(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
-    localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
-    if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
-      LOG.debug("Using local interfaces [" +
-      Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
-      Joiner.on(',').join(localInterfaceAddrs) + "]");
-    }
-    
-    Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
-        null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
-    Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
-        null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
-    Boolean writeDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
-        null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
-    this.defaultReadCachingStrategy =
-        new CachingStrategy(readDropBehind, readahead);
-    this.defaultWriteCachingStrategy =
-        new CachingStrategy(writeDropBehind, readahead);
-    this.clientContext = ClientContext.get(
-        conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
-        dfsClientConf);
-
-    if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
-      this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
-    }
-    this.saslClient = new SaslDataTransferClient(
-      conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
-      TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
-  }
-  
-  /**
-   * Return the socket addresses to use with each configured
-   * local interface. Local interfaces may be specified by IP
-   * address, IP address range using CIDR notation, interface
-   * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
-   * The socket addresses consist of the IPs for the interfaces
-   * and the ephemeral port (port 0). If an IP, IP range, or
-   * interface name matches an interface with sub-interfaces
-   * only the IP of the interface is used. Sub-interfaces can
-   * be used by specifying them explicitly (by IP or name).
-   * 
-   * @return SocketAddresses for the configured local interfaces,
-   *    or an empty array if none are configured
-   * @throws UnknownHostException if a given interface name is invalid
-   */
-  private static SocketAddress[] getLocalInterfaceAddrs(
-      String interfaceNames[]) throws UnknownHostException {
-    List<SocketAddress> localAddrs = new ArrayList<SocketAddress>();
-    for (String interfaceName : interfaceNames) {
-      if (InetAddresses.isInetAddress(interfaceName)) {
-        localAddrs.add(new InetSocketAddress(interfaceName, 0));
-      } else if (NetUtils.isValidSubnet(interfaceName)) {
-        for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
-          localAddrs.add(new InetSocketAddress(addr, 0));
-        }
-      } else {
-        for (String ip : DNS.getIPs(interfaceName, false)) {
-          localAddrs.add(new InetSocketAddress(ip, 0));
-        }
-      }
-    }
-    return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
-  }
-
-  /**
-   * Select one of the configured local interfaces at random. We use a random
-   * interface because other policies like round-robin are less effective
-   * given that we cache connections to datanodes.
-   *
-   * @return one of the local interface addresses at random, or null if no
-   *    local interfaces are configured
-   */
-  SocketAddress getRandomLocalInterfaceAddr() {
-    if (localInterfaceAddrs.length == 0) {
-      return null;
-    }
-    final int idx = r.nextInt(localInterfaceAddrs.length);
-    final SocketAddress addr = localInterfaceAddrs[idx];
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using local interface " + addr);
-    }
-    return addr;
-  }
-
-  /**
-   * Return the timeout that clients should use when writing to datanodes.
-   * @param numNodes the number of nodes in the pipeline.
-   */
-  int getDatanodeWriteTimeout(int numNodes) {
-    final int t = dfsClientConf.getDatanodeSocketWriteTimeout();
-    return t > 0? t + HdfsConstants.WRITE_TIMEOUT_EXTENSION*numNodes: 0;
-  }
-
-  int getDatanodeReadTimeout(int numNodes) {
-    final int t = dfsClientConf.getSocketTimeout();
-    return t > 0? HdfsConstants.READ_TIMEOUT_EXTENSION*numNodes + t: 0;
-  }
-  
-  @VisibleForTesting
-  public String getClientName() {
-    return clientName;
-  }
-
-  void checkOpen() throws IOException {
-    if (!clientRunning) {
-      IOException result = new IOException("Filesystem closed");
-      throw result;
-    }
-  }
-
-  /** Return the lease renewer instance. The renewer thread won't start
-   *  until the first output stream is created. The same instance will
-   *  be returned until all output streams are closed.
-   */
-  public LeaseRenewer getLeaseRenewer() throws IOException {
-      return LeaseRenewer.getInstance(authority, ugi, this);
-  }
-
-  /** Get a lease and start automatic renewal */
-  private void beginFileLease(final long inodeId, final DFSOutputStream out)
-      throws IOException {
-    getLeaseRenewer().put(inodeId, out, this);
-  }
-
-  /** Stop renewal of lease for the file. */
-  void endFileLease(final long inodeId) throws IOException {
-    getLeaseRenewer().closeFile(inodeId, this);
-  }
-    
-
-  /** Put a file. Only called from LeaseRenewer, where proper locking is
-   *  enforced to consistently update its local dfsclients array and 
-   *  client's filesBeingWritten map.
-   */
-  public void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
-    synchronized(filesBeingWritten) {
-      filesBeingWritten.put(inodeId, out);
-      // update the last lease renewal time only when there was no
-      // writes. once there is one write stream open, the lease renewer
-      // thread keeps it updated well with in anyone's expiration time.
-      if (lastLeaseRenewal == 0) {
-        updateLastLeaseRenewal();
-      }
-    }
-  }
-
-  /** Remove a file. Only called from LeaseRenewer. */
-  public void removeFileBeingWritten(final long inodeId) {
-    synchronized(filesBeingWritten) {
-      filesBeingWritten.remove(inodeId);
-      if (filesBeingWritten.isEmpty()) {
-        lastLeaseRenewal = 0;
-      }
-    }
-  }
-
-  /** Is file-being-written map empty? */
-  public boolean isFilesBeingWrittenEmpty() {
-    synchronized(filesBeingWritten) {
-      return filesBeingWritten.isEmpty();
-    }
-  }
-  
-  /** @return true if the client is running */
-  public boolean isClientRunning() {
-    return clientRunning;
-  }
-
-  long getLastLeaseRenewal() {
-    return lastLeaseRenewal;
-  }
-
-  void updateLastLeaseRenewal() {
-    synchronized(filesBeingWritten) {
-      if (filesBeingWritten.isEmpty()) {
-        return;
-      }
-      lastLeaseRenewal = Time.monotonicNow();
-    }
-  }
-
-  /**
-   * Renew leases.
-   * @return true if lease was renewed. May return false if this
-   * client has been closed or has no files open.
-   **/
-  public boolean renewLease() throws IOException {
-    if (clientRunning && !isFilesBeingWrittenEmpty()) {
-      try {
-        namenode.renewLease(clientName);
-        updateLastLeaseRenewal();
-        return true;
-      } catch (IOException e) {
-        // Abort if the lease has already expired. 
-        final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
-        if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
-          LOG.warn("Failed to renew lease for " + clientName + " for "
-              + (elapsed/1000) + " seconds (>= hard-limit ="
-              + (HdfsConstants.LEASE_HARDLIMIT_PERIOD / 1000) + " seconds.) "
-              + "Closing all files being written ...", e);
-          closeAllFilesBeingWritten(true);
-        } else {
-          // Let the lease renewer handle it and retry.
-          throw e;
-        }
-      }
-    }
-    return false;
-  }
-  
-  /**
-   * Close connections the Namenode.
-   */
-  void closeConnectionToNamenode() {
-    RPC.stopProxy(namenode);
-  }
-
-  /** Close/abort all files being written. */
-  public void closeAllFilesBeingWritten(final boolean abort) {
-    for(;;) {
-      final long inodeId;
-      final DFSOutputStream out;
-      synchronized(filesBeingWritten) {
-        if (filesBeingWritten.isEmpty()) {
-          return;
-        }
-        inodeId = filesBeingWritten.keySet().iterator().next();
-        out = filesBeingWritten.remove(inodeId);
-      }
-      if (out != null) {
-        try {
-          if (abort) {
-            out.abort();
-          } else {
-            out.close();
-          }
-        } catch(IOException ie) {
-          LOG.error("Failed to " + (abort ? "abort" : "close") + " file: "
-              + out.getSrc() + " with inode: " + inodeId, ie);
-        }
-      }
-    }
-  }
-
-  /**
-   * Close the file system, abandoning all of the leases and files being
-   * created and close connections to the namenode.
-   */
-  @Override
-  public synchronized void close() throws IOException {
-    if(clientRunning) {
-      closeAllFilesBeingWritten(false);
-      clientRunning = false;
-      getLeaseRenewer().closeClient(this);
-      // close connections to the namenode
-      closeConnectionToNamenode();
-    }
-  }
-
-  /**
-   * Close all open streams, abandoning all of the leases and files being
-   * created.
-   * @param abort whether streams should be gracefully closed
-   */
-  public void closeOutputStreams(boolean abort) {
-    if (clientRunning) {
-      closeAllFilesBeingWritten(abort);
-    }
-  }
-
-  /**
-   * @see ClientProtocol#getPreferredBlockSize(String)
-   */
-  public long getBlockSize(String f) throws IOException {
-    TraceScope scope = getPathTraceScope("getBlockSize", f);
-    try {
-      return namenode.getPreferredBlockSize(f);
-    } catch (IOException ie) {
-      LOG.warn("Problem getting block size", ie);
-      throw ie;
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Get server default values for a number of configuration params.
-   * @see ClientProtocol#getServerDefaults()
-   */
-  public FsServerDefaults getServerDefaults() throws IOException {
-    long now = Time.monotonicNow();
-    if ((serverDefaults == null) ||
-        (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
-      serverDefaults = namenode.getServerDefaults();
-      serverDefaultsLastUpdate = now;
-    }
-    assert serverDefaults != null;
-    return serverDefaults;
-  }
-  
-  /**
-   * Get a canonical token service name for this client's tokens.  Null should
-   * be returned if the client is not using tokens.
-   * @return the token service for the client
-   */
-  @InterfaceAudience.LimitedPrivate( { "HDFS" }) 
-  public String getCanonicalServiceName() {
-    return (dtService != null) ? dtService.toString() : null;
-  }
-  
-  /**
-   * @see ClientProtocol#getDelegationToken(Text)
-   */
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
-      throws IOException {
-    assert dtService != null;
-    TraceScope scope = Trace.startSpan("getDelegationToken", traceSampler);
-    try {
-      Token<DelegationTokenIdentifier> token =
-        namenode.getDelegationToken(renewer);
-      if (token != null) {
-        token.setService(this.dtService);
-        LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
-      } else {
-        LOG.info("Cannot get delegation token from " + renewer);
-      }
-      return token;
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Renew a delegation token
-   * @param token the token to renew
-   * @return the new expiration time
-   * @throws InvalidToken
-   * @throws IOException
-   * @deprecated Use Token.renew instead.
-   */
-  @Deprecated
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws InvalidToken, IOException {
-    LOG.info("Renewing " + DelegationTokenIdentifier.stringifyToken(token));
-    try {
-      return token.renew(conf);
-    } catch (InterruptedException ie) {                                       
-      throw new RuntimeException("caught interrupted", ie);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(InvalidToken.class,
-                                     AccessControlException.class);
-    }
-  }
-  
-  /**
-   * Cancel a delegation token
-   * @param token the token to cancel
-   * @throws InvalidToken
-   * @throws IOException
-   * @deprecated Use Token.cancel instead.
-   */
-  @Deprecated
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws InvalidToken, IOException {
-    LOG.info("Cancelling " + DelegationTokenIdentifier.stringifyToken(token));
-    try {
-      token.cancel(conf);
-     } catch (InterruptedException ie) {                                       
-      throw new RuntimeException("caught interrupted", ie);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(InvalidToken.class,
-                                     AccessControlException.class);
-    }
-  }
-  
-  @InterfaceAudience.Private
-  public static class Renewer extends TokenRenewer {
-    
-    static {
-      //Ensure that HDFS Configuration files are loaded before trying to use
-      // the renewer.
-      HdfsConfiguration.init();
-    }
-    
-    @Override
-    public boolean handleKind(Text kind) {
-      return DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public long renew(Token<?> token, Configuration conf) throws IOException {
-      Token<DelegationTokenIdentifier> delToken = 
-        (Token<DelegationTokenIdentifier>) token;
-      ClientProtocol nn = getNNProxy(delToken, conf);
-      try {
-        return nn.renewDelegationToken(delToken);
-      } catch (RemoteException re) {
-        throw re.unwrapRemoteException(InvalidToken.class, 
-                                       AccessControlException.class);
-      }
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public void cancel(Token<?> token, Configuration conf) throws IOException {
-      Token<DelegationTokenIdentifier> delToken = 
-          (Token<DelegationTokenIdentifier>) token;
-      LOG.info("Cancelling " + 
-               DelegationTokenIdentifier.stringifyToken(delToken));
-      ClientProtocol nn = getNNProxy(delToken, conf);
-      try {
-        nn.cancelDelegationToken(delToken);
-      } catch (RemoteException re) {
-        throw re.unwrapRemoteException(InvalidToken.class,
-            AccessControlException.class);
-      }
-    }
-    
-    private static ClientProtocol getNNProxy(
-        Token<DelegationTokenIdentifier> token, Configuration conf)
-        throws IOException {
-      URI uri = HAUtilClient.getServiceUriFromToken(
-          HdfsConstants.HDFS_URI_SCHEME, token);
-      if (HAUtilClient.isTokenForLogicalUri(token) &&
-          !HAUtilClient.isLogicalUri(conf, uri)) {
-        // If the token is for a logical nameservice, but the configuration
-        // we have disagrees about that, we can't actually renew it.
-        // This can be the case in MR, for example, if the RM doesn't
-        // have all of the HA clusters configured in its configuration.
-        throw new IOException("Unable to map logical nameservice URI '" +
-            uri + "' to a NameNode. Local configuration does not have " +
-            "a failover proxy provider configured.");
-      }
-      
-      ProxyAndInfo<ClientProtocol> info =
-        NameNodeProxiesClient.createProxyWithClientProtocol(conf, uri, null);
-      assert info.getDelegationTokenService().equals(token.getService()) :
-        "Returned service '" + info.getDelegationTokenService().toString() +
-        "' doesn't match expected service '" +
-        token.getService().toString() + "'";
-        
-      return info.getProxy();
-    }
-
-    @Override
-    public boolean isManaged(Token<?> token) throws IOException {
-      return true;
-    }
-    
-  }
-
-  /**
-   * Report corrupt blocks that were discovered by the client.
-   * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
-   */
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    namenode.reportBadBlocks(blocks);
-  }
-  
-  public LocatedBlocks getLocatedBlocks(String src, long start)
-      throws IOException {
-    return getLocatedBlocks(src, start, dfsClientConf.getPrefetchSize());
-  }
-
-  /*
-   * This is just a wrapper around callGetBlockLocations, but non-static so that
-   * we can stub it out for tests.
-   */
-  @VisibleForTesting
-  public LocatedBlocks getLocatedBlocks(String src, long start, long length)
-      throws IOException {
-    TraceScope scope = getPathTraceScope("getBlockLocations", src);
-    try {
-      return callGetBlockLocations(namenode, src, start, length);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * @see ClientProtocol#getBlockLocations(String, long, long)
-   */
-  static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
-      String src, long start, long length) 
-      throws IOException {
-    try {
-      return namenode.getBlockLocations(src, start, length);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     UnresolvedPathException.class);
-    }
-  }
-
-  /**
-   * Recover a file's lease
-   * @param src a file's path
-   * @return true if the file is already closed
-   * @throws IOException
-   */
-  boolean recoverLease(String src) throws IOException {
-    checkOpen();
-
-    TraceScope scope = getPathTraceScope("recoverLease", src);
-    try {
-      return namenode.recoverLease(src, clientName);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(FileNotFoundException.class,
-                                     AccessControlException.class,
-                                     UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Get block location info about file
-   * 
-   * getBlockLocations() returns a list of hostnames that store 
-   * data for a specific file region.  It returns a set of hostnames
-   * for every block within the indicated region.
-   *
-   * This function is very useful when writing code that considers
-   * data-placement when performing operations.  For example, the
-   * MapReduce system tries to schedule tasks on the same machines
-   * as the data-block the task processes. 
-   */
-  public BlockLocation[] getBlockLocations(String src, long start, 
-        long length) throws IOException, UnresolvedLinkException {
-    TraceScope scope = getPathTraceScope("getBlockLocations", src);
-    try {
-      LocatedBlocks blocks = getLocatedBlocks(src, start, length);
-      BlockLocation[] locations =  DFSUtilClient.locatedBlocks2Locations(blocks);
-      HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
-      for (int i = 0; i < locations.length; i++) {
-        hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
-      }
-      return hdfsLocations;
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Get block location information about a list of {@link HdfsBlockLocation}.
-   * Used by {@link DistributedFileSystem#getFileBlockStorageLocations(List)} to
-   * get {@link BlockStorageLocation}s for blocks returned by
-   * {@link DistributedFileSystem#getFileBlockLocations(org.apache.hadoop.fs.FileStatus, long, long)}
-   * .
-   * 
-   * This is done by making a round of RPCs to the associated datanodes, asking
-   * the volume of each block replica. The returned array of
-   * {@link BlockStorageLocation} expose this information as a
-   * {@link VolumeId}.
-   * 
-   * @param blockLocations
-   *          target blocks on which to query volume location information
-   * @return volumeBlockLocations original block array augmented with additional
-   *         volume location information for each replica.
-   */
-  public BlockStorageLocation[] getBlockStorageLocations(
-      List<BlockLocation> blockLocations) throws IOException,
-      UnsupportedOperationException, InvalidBlockTokenException {
-    if (!getConf().isHdfsBlocksMetadataEnabled()) {
-      throw new UnsupportedOperationException("Datanode-side support for " +
-          "getVolumeBlockLocations() must also be enabled in the client " +
-          "configuration.");
-    }
-    // Downcast blockLocations and fetch out required LocatedBlock(s)
-    List<LocatedBlock> blocks = new ArrayList<LocatedBlock>();
-    for (BlockLocation loc : blockLocations) {
-      if (!(loc instanceof HdfsBlockLocation)) {
-        throw new ClassCastException("DFSClient#getVolumeBlockLocations " +
-            "expected to be passed HdfsBlockLocations");
-      }
-      HdfsBlockLocation hdfsLoc = (HdfsBlockLocation) loc;
-      blocks.add(hdfsLoc.getLocatedBlock());
-    }
-    
-    // Re-group the LocatedBlocks to be grouped by datanodes, with the values
-    // a list of the LocatedBlocks on the datanode.
-    Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks = 
-        new LinkedHashMap<DatanodeInfo, List<LocatedBlock>>();
-    for (LocatedBlock b : blocks) {
-      for (DatanodeInfo info : b.getLocations()) {
-        if (!datanodeBlocks.containsKey(info)) {
-          datanodeBlocks.put(info, new ArrayList<LocatedBlock>());
-        }
-        List<LocatedBlock> l = datanodeBlocks.get(info);
-        l.add(b);
-      }
-    }
-        
-    // Make RPCs to the datanodes to get volume locations for its replicas
-    TraceScope scope =
-      Trace.startSpan("getBlockStorageLocations", traceSampler);
-    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas;
-    try {
-      metadatas = BlockStorageLocationUtil.
-          queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
-              getConf().getFileBlockStorageLocationsNumThreads(),
-              getConf().getFileBlockStorageLocationsTimeoutMs(),
-              getConf().isConnectToDnViaHostname());
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("metadata returned: "
-            + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
-      }
-    } finally {
-      scope.close();
-    }
-    
-    // Regroup the returned VolumeId metadata to again be grouped by
-    // LocatedBlock rather than by datanode
-    Map<LocatedBlock, List<VolumeId>> blockVolumeIds = BlockStorageLocationUtil
-        .associateVolumeIdsWithBlocks(blocks, metadatas);
-    
-    // Combine original BlockLocations with new VolumeId information
-    BlockStorageLocation[] volumeBlockLocations = BlockStorageLocationUtil
-        .convertToVolumeBlockLocations(blocks, blockVolumeIds);
-
-    return volumeBlockLocations;
-  }
-
-  /**
-   * Decrypts a EDEK by consulting the KeyProvider.
-   */
-  private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
-      feInfo) throws IOException {
-    TraceScope scope = Trace.startSpan("decryptEDEK", traceSampler);
-    try {
-      KeyProvider provider = getKeyProvider();
-      if (provider == null) {
-        throw new IOException("No KeyProvider is configured, cannot access" +
-            " an encrypted file");
-      }
-      EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
-          feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
-          feInfo.getEncryptedDataEncryptionKey());
-      try {
-        KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
-            .createKeyProviderCryptoExtension(provider);
-        return cryptoProvider.decryptEncryptedKey(ekv);
-      } catch (GeneralSecurityException e) {
-        throw new IOException(e);
-      }
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Obtain the crypto protocol version from the provided FileEncryptionInfo,
-   * checking to see if this version is supported by.
-   *
-   * @param feInfo FileEncryptionInfo
-   * @return CryptoProtocolVersion from the feInfo
-   * @throws IOException if the protocol version is unsupported.
-   */
-  private static CryptoProtocolVersion getCryptoProtocolVersion
-      (FileEncryptionInfo feInfo) throws IOException {
-    final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
-    if (!CryptoProtocolVersion.supports(version)) {
-      throw new IOException("Client does not support specified " +
-          "CryptoProtocolVersion " + version.getDescription() + " version " +
-          "number" + version.getVersion());
-    }
-    return version;
-  }
-
-  /**
-   * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
-   * and the available CryptoCodecs configured in the Configuration.
-   *
-   * @param conf   Configuration
-   * @param feInfo FileEncryptionInfo
-   * @return CryptoCodec
-   * @throws IOException if no suitable CryptoCodec for the CipherSuite is
-   *                     available.
-   */
-  private static CryptoCodec getCryptoCodec(Configuration conf,
-      FileEncryptionInfo feInfo) throws IOException {
-    final CipherSuite suite = feInfo.getCipherSuite();
-    if (suite.equals(CipherSuite.UNKNOWN)) {
-      throw new IOException("NameNode specified unknown CipherSuite with ID "
-          + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
-    }
-    final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
-    if (codec == null) {
-      throw new UnknownCipherSuiteException(
-          "No configuration found for the cipher suite "
-          + suite.getConfigSuffix() + " prefixed with "
-          + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
-          + ". Please see the example configuration "
-          + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
-          + "at core-default.xml for details.");
-    }
-    return codec;
-  }
-
-  /**
-   * Wraps the stream in a CryptoInputStream if the underlying file is
-   * encrypted.
-   */
-  public HdfsDataInputStream createWrappedInputStream(DFSInputStream dfsis)
-      throws IOException {
-    final FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
-    if (feInfo != null) {
-      // File is encrypted, wrap the stream in a crypto stream.
-      // Currently only one version, so no special logic based on the version #
-      getCryptoProtocolVersion(feInfo);
-      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
-      final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
-      final CryptoInputStream cryptoIn =
-          new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
-              feInfo.getIV());
-      return new HdfsDataInputStream(cryptoIn);
-    } else {
-      // No FileEncryptionInfo so no encryption.
-      return new HdfsDataInputStream(dfsis);
-    }
-  }
-
-  /**
-   * Wraps the stream in a CryptoOutputStream if the underlying file is
-   * encrypted.
-   */
-  public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
-      FileSystem.Statistics statistics) throws IOException {
-    return createWrappedOutputStream(dfsos, statistics, 0);
-  }
-
-  /**
-   * Wraps the stream in a CryptoOutputStream if the underlying file is
-   * encrypted.
-   */
-  public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
-      FileSystem.Statistics statistics, long startPos) throws IOException {
-    final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
-    if (feInfo != null) {
-      // File is encrypted, wrap the stream in a crypto stream.
-      // Currently only one version, so no special logic based on the version #
-      getCryptoProtocolVersion(feInfo);
-      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
-      KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
-      final CryptoOutputStream cryptoOut =
-          new CryptoOutputStream(dfsos, codec,
-              decrypted.getMaterial(), feInfo.getIV(), startPos);
-      return new HdfsDataOutputStream(cryptoOut, statistics, startPos);
-    } else {
-      // No FileEncryptionInfo present so no encryption.
-      return new HdfsDataOutputStream(dfsos, statistics, startPos);
-    }
-  }
-
-  public DFSInputStream open(String src) 
-      throws IOException, UnresolvedLinkException {
-    return open(src, dfsClientConf.getIoBufferSize(), true, null);
-  }
-
-  /**
-   * Create an input stream that obtains a nodelist from the
-   * namenode, and then reads from all the right places.  Creates
-   * inner subclass of InputStream that does the right out-of-band
-   * work.
-   * @deprecated Use {@link #open(String, int, boolean)} instead.
-   */
-  @Deprecated
-  public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
-                             FileSystem.Statistics stats)
-      throws IOException, UnresolvedLinkException {
-    return open(src, buffersize, verifyChecksum);
-  }
-  
-
-  /**
-   * Create an input stream that obtains a nodelist from the
-   * namenode, and then reads from all the right places.  Creates
-   * inner subclass of InputStream that does the right out-of-band
-   * work.
-   */
-  public DFSInputStream open(String src, int buffersize, boolean verifyChecksum)
-      throws IOException, UnresolvedLinkException {
-    checkOpen();
-    //    Get block info from namenode
-    TraceScope scope = getPathTraceScope("newDFSInputStream", src);
-    try {
-      return new DFSInputStream(this, src, verifyChecksum, null);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Get the namenode associated with this DFSClient object
-   * @return the namenode associated with this DFSClient object
-   */
-  public ClientProtocol getNamenode() {
-    return namenode;
-  }
-  
-  /**
-   * Call {@link #create(String, boolean, short, long, Progressable)} with
-   * default <code>replication</code> and <code>blockSize<code> and null <code>
-   * progress</code>.
-   */
-  public OutputStream create(String src, boolean overwrite) 
-      throws IOException {
-    return create(src, overwrite, dfsClientConf.getDefaultReplication(),
-        dfsClientConf.getDefaultBlockSize(), null);
-  }
-    
-  /**
-   * Call {@link #create(String, boolean, short, long, Progressable)} with
-   * default <code>replication</code> and <code>blockSize<code>.
-   */
-  public OutputStream create(String src, 
-                             boolean overwrite,
-                             Progressable progress) throws IOException {
-    return create(src, overwrite, dfsClientConf.getDefaultReplication(),
-        dfsClientConf.getDefaultBlockSize(), progress);
-  }
-    
-  /**
-   * Call {@link #create(String, boolean, short, long, Progressable)} with
-   * null <code>progress</code>.
-   */
-  public OutputStream create(String src, 
-                             boolean overwrite, 
-                             short replication,
-                             long blockSize) throws IOException {
-    return create(src, overwrite, replication, blockSize, null);
-  }
-
-  /**
-   * Call {@link #create(String, boolean, short, long, Progressable, int)}
-   * with default bufferSize.
-   */
-  public OutputStream create(String src, boolean overwrite, short replication,
-      long blockSize, Progressable progress) throws IOException {
-    return create(src, overwrite, replication, blockSize, progress,
-        dfsClientConf.getIoBufferSize());
-  }
-
-  /**
-   * Call {@link #create(String, FsPermission, EnumSet, short, long, 
-   * Progressable, int, ChecksumOpt)} with default <code>permission</code>
-   * {@link FsPermission#getFileDefault()}.
-   * 
-   * @param src File name
-   * @param overwrite overwrite an existing file if true
-   * @param replication replication factor for the file
-   * @param blockSize maximum block size
-   * @param progress interface for reporting client progress
-   * @param buffersize underlying buffersize
-   * 
-   * @return output stream
-   */
-  public OutputStream create(String src,
-                             boolean overwrite,
-                             short replication,
-                             long blockSize,
-                             Progressable progress,
-                             int buffersize)
-      throws IOException {
-    return create(src, FsPermission.getFileDefault(),
-        overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
-            : EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
-        buffersize, null);
-  }
-
-  /**
-   * Call {@link #create(String, FsPermission, EnumSet, boolean, short, 
-   * long, Progressable, int, ChecksumOpt)} with <code>createParent</code>
-   *  set to true.
-   */
-  public DFSOutputStream create(String src, 
-                             FsPermission permission,
-                             EnumSet<CreateFlag> flag, 
-                             short replication,
-                             long blockSize,
-                             Progressable progress,
-                             int buffersize,
-                             ChecksumOpt checksumOpt)
-      throws IOException {
-    return create(src, permission, flag, true,
-        replication, blockSize, progress, buffersize, checksumOpt, null);
-  }
-
-  /**
-   * Create a new dfs file with the specified block replication 
-   * with write-progress reporting and return an output stream for writing
-   * into the file.  
-   * 
-   * @param src File name
-   * @param permission The permission of the directory being created.
-   *          If null, use default permission {@link FsPermission#getFileDefault()}
-   * @param flag indicates create a new file or create/overwrite an
-   *          existing file or append to an existing file
-   * @param createParent create missing parent directory if true
-   * @param replication block replication
-   * @param blockSize maximum block size
-   * @param progress interface for reporting client progress
-   * @param buffersize underlying buffer size 
-   * @param checksumOpt checksum options
-   * 
-   * @return output stream
-   *
-   * @see ClientProtocol#create for detailed description of exceptions thrown
-   */
-  public DFSOutputStream create(String src, 
-                             FsPermission permission,
-                             EnumSet<CreateFlag> flag, 
-                             boolean createParent,
-                             short replication,
-                             long blockSize,
-                             Progressable progress,
-                             int buffersize,
-                             ChecksumOpt checksumOpt) throws IOException {
-    return create(src, permission, flag, createParent, replication, blockSize, 
-        progress, buffersize, checksumOpt, null);
-  }
-
-  private FsPermission applyUMask(FsPermission permission) {
-    if (permission == null) {
-      permission = FsPermission.getFileDefault();
-    }
-    return permission.applyUMask(dfsClientConf.getUMask());
-  }
-
-  /**
-   * Same as {@link #create(String, FsPermission, EnumSet, boolean, short, long,
-   * Progressable, int, ChecksumOpt)} with the addition of favoredNodes that is
-   * a hint to where the namenode should place the file blocks.
-   * The favored nodes hint is not persisted in HDFS. Hence it may be honored
-   * at the creation time only. HDFS could move the blocks during balancing or
-   * replication, to move the blocks from favored nodes. A value of null means
-   * no favored nodes for this create
-   */
-  public DFSOutputStream create(String src, 
-                             FsPermission permission,
-                             EnumSet<CreateFlag> flag, 
-                             boolean createParent,
-                             short replication,
-                             long blockSize,
-                             Progressable progress,
-                             int buffersize,
-                             ChecksumOpt checksumOpt,
-                             InetSocketAddress[] favoredNodes) throws IOException {
-    checkOpen();
-    final FsPermission masked = applyUMask(permission);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + masked);
-    }
-    final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
-        src, masked, flag, createParent, replication, blockSize, progress,
-        buffersize, dfsClientConf.createChecksum(checksumOpt),
-        getFavoredNodesStr(favoredNodes));
-    beginFileLease(result.getFileId(), result);
-    return result;
-  }
-
-  private String[] getFavoredNodesStr(InetSocketAddress[] favoredNodes) {
-    String[] favoredNodeStrs = null;
-    if (favoredNodes != null) {
-      favoredNodeStrs = new String[favoredNodes.length];
-      for (int i = 0; i < favoredNodes.length; i++) {
-        favoredNodeStrs[i] = 
-            favoredNodes[i].getHostName() + ":" 
-                         + favoredNodes[i].getPort();
-      }
-    }
-    return favoredNodeStrs;
-  }
-  
-  /**
-   * Append to an existing file if {@link CreateFlag#APPEND} is present
-   */
-  private DFSOutputStream primitiveAppend(String src, EnumSet<CreateFlag> flag,
-      int buffersize, Progressable progress) throws IOException {
-    if (flag.contains(CreateFlag.APPEND)) {
-      HdfsFileStatus stat = getFileInfo(src);
-      if (stat == null) { // No file to append to
-        // New file needs to be created if create option is present
-        if (!flag.contains(CreateFlag.CREATE)) {
-          throw new FileNotFoundException("failed to append to non-existent file "
-              + src + " on client " + clientName);
-        }
-        return null;
-      }
-      return callAppend(src, buffersize, flag, progress, null);
-    }
-    return null;
-  }
-  
-  /**
-   * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
-   *  Progressable, int, ChecksumOpt)} except that the permission
-   *  is absolute (ie has already been masked with umask.
-   */
-  public DFSOutputStream primitiveCreate(String src, 
-                             FsPermission absPermission,
-                             EnumSet<CreateFlag> flag,
-                             boolean createParent,
-                             short replication,
-                             long blockSize,
-                             Progressable progress,
-                             int buffersize,
-                             ChecksumOpt checksumOpt)
-      throws IOException, UnresolvedLinkException {
-    checkOpen();
-    CreateFlag.validate(flag);
-    DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress);
-    if (result == null) {
-      DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt);
-      result = DFSOutputStream.newStreamForCreate(this, src, absPermission,
-          flag, createParent, replication, blockSize, progress, buffersize,
-          checksum, null);
-    }
-    beginFileLease(result.getFileId(), result);
-    return result;
-  }
-  
-  /**
-   * Creates a symbolic link.
-   * 
-   * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean) 
-   */
-  public void createSymlink(String target, String link, boolean createParent)
-      throws IOException {
-    TraceScope scope = getPathTraceScope("createSymlink", target);
-    try {
-      final FsPermission dirPerm = applyUMask(null);
-      namenode.createSymlink(target, link, dirPerm, createParent);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileAlreadyExistsException.class, 
-                                     FileNotFoundException.class,
-                                     ParentNotDirectoryException.class,
-                                     NSQuotaExceededException.class, 
-                                     DSQuotaExceededException.class,
-                                     QuotaByStorageTypeExceededException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Resolve the *first* symlink, if any, in the path.
-   * 
-   * @see ClientProtocol#getLinkTarget(String)
-   */
-  public String getLinkTarget(String path) throws IOException { 
-    checkOpen();
-    TraceScope scope = getPathTraceScope("getLinkTarget", path);
-    try {
-      return namenode.getLinkTarget(path);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /** Method to get stream returned by append call */
-  private DFSOutputStream callAppend(String src, int buffersize,
-      EnumSet<CreateFlag> flag, Progressable progress, String[] favoredNodes)
-      throws IOException {
-    CreateFlag.validateForAppend(flag);
-    try {
-      LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
-          new EnumSetWritable<>(flag, CreateFlag.class));
-      return DFSOutputStream.newStreamForAppend(this, src, flag, buffersize,
-          progress, blkWithStatus.getLastBlock(),
-          blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(null),
-          favoredNodes);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     SafeModeException.class,
-                                     DSQuotaExceededException.class,
-                                     QuotaByStorageTypeExceededException.class,
-                                     UnsupportedOperationException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    }
-  }
-  
-  /**
-   * Append to an existing HDFS file.  
-   * 
-   * @param src file name
-   * @param buffersize buffer size
-   * @param flag indicates whether to append data to a new block instead of
-   *             the last block
-   * @param progress for reporting write-progress; null is acceptable.
-   * @param statistics file system statistics; null is acceptable.
-   * @return an output stream for writing into the file
-   * 
-   * @see ClientProtocol#append(String, String, EnumSetWritable)
-   */
-  public HdfsDataOutputStream append(final String src, final int buffersize,
-      EnumSet<CreateFlag> flag, final Progressable progress,
-      final FileSystem.Statistics statistics) throws IOException {
-    final DFSOutputStream out = append(src, buffersize, flag, null, progress);
-    return createWrappedOutputStream(out, statistics, out.getInitialLen());
-  }
-
-  /**
-   * Append to an existing HDFS file.
-   * 
-   * @param src file name
-   * @param buffersize buffer size
-   * @param flag indicates whether to append data to a new block instead of the
-   *          last block
-   * @param progress for reporting write-progress; null is acceptable.
-   * @param statistics file system statistics; null is acceptable.
-   * @param favoredNodes FavoredNodes for new blocks
-   * @return an output stream for writing into the file
-   * @see ClientProtocol#append(String, String, EnumSetWritable)
-   */
-  public HdfsDataOutputStream append(final String src, final int buffersize,
-      EnumSet<CreateFlag> flag, final Progressable progress,
-      final FileSystem.Statistics statistics,
-      final InetSocketAddress[] favoredNodes) throws IOException {
-    final DFSOutputStream out = append(src, buffersize, flag,
-        getFavoredNodesStr(favoredNodes), progress);
-    return createWrappedOutputStream(out, statistics, out.getInitialLen());
-  }
-
-  private DFSOutputStream append(String src, int buffersize,
-      EnumSet<CreateFlag> flag, String[] favoredNodes, Progressable progress)
-      throws IOException {
-    checkOpen();
-    final DFSOutputStream result = callAppend(src, buffersize, flag, progress,
-        favoredNodes);
-    beginFileLease(result.getFileId(), result);
-    return result;
-  }
-
-  /**
-   * Set replication for an existing file.
-   * @param src file name
-   * @param replication replication to set the file to
-   * 
-   * @see ClientProtocol#setReplication(String, short)
-   */
-  public boolean setReplication(String src, short replication)
-      throws IOException {
-    TraceScope scope = getPathTraceScope("setReplication", src);
-    try {
-      return namenode.setReplication(src, replication);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     SafeModeException.class,
-                                     DSQuotaExceededException.class,
-                                     QuotaByStorageTypeExceededException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Set storage policy for an existing file/directory
-   * @param src file/directory name
-   * @param policyName name of the storage policy
-   */
-  public void setStoragePolicy(String src, String policyName)
-      throws IOException {
-    TraceScope scope = getPathTraceScope("setStoragePolicy", src);
-    try {
-      namenode.setStoragePolicy(src, policyName);
-    } catch (RemoteException e) {
-      throw e.unwrapRemoteException(AccessControlException.class,
-                                    FileNotFoundException.class,
-                                    SafeModeException.class,
-                                    NSQuotaExceededException.class,
-                                    UnresolvedPathException.class,
-                                    SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * @param path file/directory name
-   * @return Get the storage policy for specified path
-   */
-  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("getStoragePolicy", path);
-    try {
-      return namenode.getStoragePolicy(path);
-    } catch (RemoteException e) {
-      throw e.unwrapRemoteException(AccessControlException.class,
-                                    FileNotFoundException.class,
-                                    SafeModeException.class,
-                                    UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * @return All the existing storage policies
-   */
-  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler);
-    try {
-      return namenode.getStoragePolicies();
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Rename file or directory.
-   * @see ClientProtocol#rename(String, String)
-   * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
-   */
-  @Deprecated
-  public boolean rename(String src, String dst) throws IOException {
-    checkOpen();
-    TraceScope scope = getSrcDstTraceScope("rename", src, dst);
-    try {
-      return namenode.rename(src, dst);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     NSQuotaExceededException.class,
-                                     DSQuotaExceededException.class,
-                                     QuotaByStorageTypeExceededException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Move blocks from src to trg and delete src
-   * See {@link ClientProtocol#concat}.
-   */
-  public void concat(String trg, String [] srcs) throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("concat", traceSampler);
-    try {
-      namenode.concat(trg, srcs);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-  /**
-   * Rename file or directory.
-   * @see ClientProtocol#rename2(String, String, Options.Rename...)
-   */
-  public void rename(String src, String dst, Options.Rename... options)
-      throws IOException {
-    checkOpen();
-    TraceScope scope = getSrcDstTraceScope("rename2", src, dst);
-    try {
-      namenode.rename2(src, dst, options);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     DSQuotaExceededException.class,
-                                     QuotaByStorageTypeExceededException.class,
-                                     FileAlreadyExistsException.class,
-                                     FileNotFoundException.class,
-                                     ParentNotDirectoryException.class,
-                                     SafeModeException.class,
-                                     NSQuotaExceededException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Truncate a file to an indicated size
-   * See {@link ClientProtocol#truncate}.
-   */
-  public boolean truncate(String src, long newLength) throws IOException {
-    checkOpen();
-    if (newLength < 0) {
-      throw new HadoopIllegalArgumentException(
-          "Cannot truncate to a negative file size: " + newLength + ".");
-    }
-    try {
-      return namenode.truncate(src, newLength, clientName);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-          UnresolvedPathException.class);
-    }
-  }
-
-  /**
-   * Delete file or directory.
-   * See {@link ClientProtocol#delete(String, boolean)}. 
-   */
-  @Deprecated
-  public boolean delete(String src) throws IOException {
-    checkOpen();
-    return delete(src, true);
-  }
-
-  /**
-   * delete file or directory.
-   * delete contents of the directory if non empty and recursive 
-   * set to true
-   *
-   * @see ClientProtocol#delete(String, boolean)
-   */
-  public boolean delete(String src, boolean recursive) throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("delete", src);
-    try {
-      return namenode.delete(src, recursive);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     SafeModeException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /** Implemented using getFileInfo(src)
-   */
-  public boolean exists(String src) throws IOException {
-    checkOpen();
-    return getFileInfo(src) != null;
-  }
-
-  /**
-   * Get a partial listing of the indicated directory
-   * No block locations need to be fetched
-   */
-  public DirectoryListing listPaths(String src,  byte[] startAfter)
-    throws IOException {
-    return listPaths(src, startAfter, false);
-  }
-  
-  /**
-   * Get a partial listing of the indicated directory
-   *
-   * Recommend to use HdfsFileStatus.EMPTY_NAME as startAfter
-   * if the application wants to fetch a listing starting from
-   * the first entry in the directory
-   *
-   * @see ClientProtocol#getListing(String, byte[], boolean)
-   */
-  public DirectoryListing listPaths(String src,  byte[] startAfter,
-      boolean needLocation) throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("listPaths", src);
-    try {
-      return namenode.getListing(src, startAfter, needLocation);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Get the file info for a specific file or directory.
-   * @param src The string representation of the path to the file
-   * @return object containing information regarding the file
-   *         or null if file not found
-   *         
-   * @see ClientProtocol#getFileInfo(String) for description of exceptions
-   */
-  public HdfsFileStatus getFileInfo(String src) throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("getFileInfo", src);
-    try {
-      return namenode.getFileInfo(src);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Close status of a file
-   * @return true if file is already closed
-   */
-  public boolean isFileClosed(String src) throws IOException{
-    checkOpen();
-    TraceScope scope = getPathTraceScope("isFileClosed", src);
-    try {
-      return namenode.isFileClosed(src);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Get the file info for a specific file or directory. If src
-   * refers to a symlink then the FileStatus of the link is returned.
-   * @param src path to a file or directory.
-   * 
-   * For description of exceptions thrown 
-   * @see ClientProtocol#getFileLinkInfo(String)
-   */
-  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("getFileLinkInfo", src);
-    try {
-      return namenode.getFileLinkInfo(src);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-   }
-  
-  @InterfaceAudience.Private
-  public void clearDataEncryptionKey() {
-    LOG.debug("Clearing encryption key");
-    synchronized (this) {
-      encryptionKey = null;
-    }
-  }
-  
-  /**
-   * @return true if data sent between this client and DNs should be encrypted,
-   *         false otherwise.
-   * @throws IOException in the event of error communicating with the NN
-   */
-  boolean shouldEncryptData() throws IOException {
-    FsServerDefaults d = getServerDefaults();
-    return d == null ? false : d.getEncryptDataTransfer();
-  }
-  
-  @Override
-  public DataEncryptionKey newDataEncryptionKey() throws IOException {
-    if (shouldEncryptData()) {
-      synchronized (this) {
-        if (encryptionKey == null ||
-            encryptionKey.expiryDate < Time.now()) {
-          LOG.debug("Getting new encryption token from NN");
-          encryptionKey = namenode.getDataEncryptionKey();
-        }
-        return encryptionKey;
-      }
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * Get the checksum of the whole file of a range of the file. Note that the
-   * range always starts from the beginning of the file.
-   * @param src The file path
-   * @param length the length of the range, i.e., the range is [0, length]
-   * @return The checksum 
-   * @see DistributedFileSystem#getFileChecksum(Path)
-   */
-  public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
-      throws IOException {
-    checkOpen();
-    Preconditions.checkArgument(length >= 0);
-    //get block locations for the file range
-    LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0,
-        length);
-    if (null == blockLocations) {
-      throw new FileNotFoundException("File does not exist: " + src);
-    }
-    if (blockLocations.isUnderConstruction()) {
-      throw new IOException("Fail to get checksum, since file " + src
-          + " is under construction.");
-    }
-    List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
-    final DataOutputBuffer md5out = new DataOutputBuffer();
-    int bytesPerCRC = -1;
-    DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
-    long crcPerBlock = 0;
-    boolean refetchBlocks = false;
-    int lastRetriedIndex = -1;
-
-    // get block checksum for each block
-    long remaining = length;
-    if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
-      remaining = Math.min(length, blockLocations.getFileLength());
-    }
-    for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
-      if (refetchBlocks) {  // refetch to get fresh tokens
-        blockLocations = callGetBlockLocations(namenode, src, 0, length);
-        if (null == blockLocations) {
-          throw new FileNotFoundException("File does not exist: " + src);
-        }
-        if (blockLocations.isUnderConstruction()) {
-          throw new IOException("Fail to get checksum, since file " + src
-              + " is under construction.");
-        }
-        locatedblocks = blockLocations.getLocatedBlocks();
-        refetchBlocks = false;
-      }
-      LocatedBlock lb = locatedblocks.get(i);
-      final ExtendedBlock block = lb.getBlock();
-      if (remaining < block.getNumBytes()) {
-        block.setNumBytes(remaining);
-      }
-      remaining -= block.getNumBytes();
-      final DatanodeInfo[] datanodes = lb.getLocations();
-      
-      //try each datanode location of the block
-      final int timeout = 3000*datanodes.length + dfsClientConf.getSocketTimeout();
-      boolean done = false;
-      for(int j = 0; !done && j < datanodes.length; j++) {
-        DataOutputStream out = null;
-        DataInputStream in = null;
-        
-        try {
-          //connect to a datanode
-          IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
-          out = new DataOutputStream(new BufferedOutputStream(pair.out,
-              smallBufferSize));
-          in = new DataInputStream(pair.in);
-
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("write to " + datanodes[j] + ": "
-                + Op.BLOCK_CHECKSUM + ", block=" + block);
-          }
-          // get block MD5
-          new Sender(out).blockChecksum(block, lb.getBlockToken());
-
-          final BlockOpResponseProto reply =
-            BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
-
-          String logInfo = "for block " + block + " from datanode " + datanodes[j];
-          DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
-
-          OpBlockChecksumResponseProto checksumData =
-            reply.getChecksumResponse();
-
-          //read byte-per-checksum
-          final int bpc = checksumData.getBytesPerCrc();
-          if (i == 0) { //first block
-            bytesPerCRC = bpc;
-          }
-          else if (bpc != bytesPerCRC) {
-            throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
-                + " but bytesPerCRC=" + bytesPerCRC);
-          }
-          
-          //read crc-per-block
-          final long cpb = checksumData.getCrcPerBlock();
-          if (locatedblocks.size() > 1 && i == 0) {
-            crcPerBlock = cpb;
-          }
-
-          //read md5
-          final MD5Hash md5 = new MD5Hash(
-              checksumData.getMd5().toByteArray());
-          md5.write(md5out);
-          
-          // read crc-type
-          final DataChecksum.Type ct;
-          if (checksumData.hasCrcType()) {
-            ct = PBHelperClient.convert(checksumData
-                .getCrcType());
-          } else {
-            LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
-                      "inferring checksum by reading first byte");
-            ct = inferChecksumTypeByReading(lb, datanodes[j]);
-          }
-
-          if (i == 0) { // first block
-            crcType = ct;
-          } else if (crcType != DataChecksum.Type.MIXED
-              && crcType != ct) {
-            // if crc types are mixed in a file
-            crcType = DataChecksum.Type.MIXED;
-          }
-
-          done = true;
-
-          if (LOG.isDebugEnabled()) {
-            if (i == 0) {
-              LOG.debug("set bytesPerCRC=" + bytesPerCRC
-                  + ", crcPerBlock=" + crcPerBlock);
-            }
-            LOG.debug("got reply from " + datanodes[j] + ": md5=" + md5);
-          }
-        } catch (InvalidBlockTokenException ibte) {
-          if (i > lastRetriedIndex) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
-                  + "for file " + src + " for block " + block
-                  + " from datanode " + datanodes[j]
-                  + ". Will retry the block once.");
-            }
-            lastRetriedIndex = i;
-            done = true; // actually it's not done; but we'll retry
-            i--; // repeat at i-th block
-            refetchBlocks = true;
-            break;
-          }
-        } catch (IOException ie) {
-          LOG.warn("src=" + src + ", datanodes["+j+"]=" + datanodes[j], ie);
-        } finally {
-          IOUtils.closeStream(in);
-          IOUtils.closeStream(out);
-        }
-      }
-
-      if (!done) {
-        throw new IOException("Fail to get block MD5 for " + block);
-      }
-    }
-
-    //compute file MD5
-    final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData()); 
-    switch (crcType) {
-      case CRC32:
-        return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
-            crcPerBlock, fileMD5);
-      case CRC32C:
-        return new MD5MD5CRC32CastagnoliFileChecksum(bytesPerCRC,
-            crcPerBlock, fileMD5);
-      default:
-        // If there is no block allocated for the file,
-        // return one with the magic entry that matches what previous
-        // hdfs versions return.
-        if (locatedblocks.size() == 0) {
-          return new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
-        }
-
-        // we should never get here since the validity was checked
-        // when getCrcType() was called above.
-        return null;
-    }
-  }
-
-  /**
-   * Connect to the given datanode's datantrasfer port, and return
-   * the resulting IOStreamPair. This includes encryption wrapping, etc.
-   */
-  private IOStreamPair connectToDN(DatanodeInfo dn, int timeout,
-      LocatedBlock lb) throws IOException {
-    boolean success = false;
-    Socket sock = null;
-    try {
-      sock = socketFactory.createSocket();
-      String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Connecting to datanode " + dnAddr);
-      }
-      NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
-      sock.setSoTimeout(timeout);
-  
-      OutputStream unbufOut = NetUtils.getOutputStream(sock);
-      InputStream unbufIn = NetUtils.getInputStream(sock);
-      IOStreamPair ret = saslClient.newSocketSend(sock, unbufOut, unbufIn, this,
-        lb.getBlockToken(), dn);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        IOUtils.closeSocket(sock);
-      }
-    }
-  }
-  
-  /**
-   * Infer the checksum type for a replica by sending an OP_READ_BLOCK
-   * for the first byte of that replica. This is used for compatibility
-   * with older HDFS versions which did not include the checksum type in
-   * OpBlockChecksumResponseProto.
-   *
-   * @param lb the located block
-   * @param dn the connected datanode
-   * @return the inferred checksum type
-   * @throws IOException if an error occurs
-   */
-  private Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
-      throws IOException {
-    IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(), lb);
-
-    try {
-      DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
-          smallBufferSize));
-      DataInputStream in = new DataInputStream(pair.in);
-  
-      new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
-          0, 1, true, CachingStrategy.newDefaultStrategy());
-      final BlockOpResponseProto reply =
-          BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
-      String logInfo = "trying to read " + lb.getBlock() + " from datanode " + dn;
-      DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
-
-      return PBHelperClient.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
-    } finally {
-      IOUtils.cleanup(null, pair.in, pair.out);
-    }
-  }
-
-  /**
-   * Set permissions to a file or directory.
-   * @param src path name.
-   * @param permission permission to set to
-   * 
-   * @see ClientProtocol#setPermission(String, FsPermission)
-   */
-  public void setPermission(String src, FsPermission permission)
-      throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("setPermission", src);
-    try {
-      namenode.setPermission(src, permission);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     SafeModeException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Set file or directory owner.
-   * @param src path name.
-   * @param username user id.
-   * @param groupname user group.
-   * 
-   * @see ClientProtocol#setOwner(String, String, String)
-   */
-  public void setOwner(String src, String username, String groupname)
-      throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("setOwner", src);
-    try {
-      namenode.setOwner(src, username, groupname);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     SafeModeException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class);                                   
-    } finally {
-      scope.close();
-    }
-  }
-
-  private long[] callGetStats() throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("getStats", traceSampler);
-    try {
-      return namenode.getStats();
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * @see ClientProtocol#getStats()
-   */
-  public FsStatus getDiskStatus() throws IOException {
-    long rawNums[] = callGetStats();
-    return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
-  }
-
-  /**
-   * Returns count of blocks with no good replicas left. Normally should be 
-   * zero.
-   * @throws IOException
-   */ 
-  public long getMissingBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
-  }
-  
-  /**
-   * Returns count of blocks with replication factor 1 and have
-   * lost the only replica.
-   * @throws IOException
-   */
-  public long getMissingReplOneBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.
-        GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
-  }
-
-  /**
-   * Returns count of blocks with one of more replica missing.
-   * @throws IOException
-   */ 
-  public long getUnderReplicatedBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
-  }
-  
-  /**
-   * Returns count of blocks with at least one replica marked corrupt. 
-   * @throws IOException
-   */ 
-  public long getCorruptBlocksCount() throws IOException {
-    return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
-  }
-  
-  /**
-   * @return a list in which each entry describes a corrupt file/block
-   * @throws IOException
-   */
-  public CorruptFileBlocks listCorruptFileBlocks(String path,
-                                                 String cookie)
-        throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("listCorruptFileBlocks", path);
-    try {
-      return namenode.listCorruptFileBlocks(path, cookie);
-    } finally {
-      scope.close();
-    }
-  }
-
-  public DatanodeInfo[] datanodeReport(DatanodeReportType type)
-      throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("datanodeReport", traceSampler);
-    try {
-      return namenode.getDatanodeReport(type);
-    } finally {
-      scope.close();
-    }
-  }
-    
-  public DatanodeStorageReport[] getDatanodeStorageReport(
-      DatanodeReportType type) throws IOException {
-    checkOpen();
-    TraceScope scope =
-        Trace.startSpan("datanodeStorageReport", traceSampler);
-    try {
-      return namenode.getDatanodeStorageReport(type);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Enter, leave or get safe mode.
-   * 
-   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,boolean)
-   */
-  public boolean setSafeMode(SafeModeAction action) throws IOException {
-    return setSafeMode(action, false);
-  }
-  
-  /**
-   * Enter, leave or get safe mode.
-   * 
-   * @param action
-   *          One of SafeModeAction.GET, SafeModeAction.ENTER and
-   *          SafeModeActiob.LEAVE
-   * @param isChecked
-   *          If true, then check only active namenode's safemode status, else
-   *          check first namenode's status.
-   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
-   */
-  public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
-    TraceScope scope =
-        Trace.startSpan("setSafeMode", traceSampler);
-    try {
-      return namenode.setSafeMode(action, isChecked);
-    } finally {
-      scope.close();
-    }
-  }
- 
-  /**
-   * Create one snapshot.
-   * 
-   * @param snapshotRoot The directory where the snapshot is to be taken
-   * @param snapshotName Name of the snapshot
-   * @return the snapshot path.
-   * @see ClientProtocol#createSnapshot(String, String)
-   */
-  public String createSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("createSnapshot", traceSampler);
-    try {
-      return namenode.createSnapshot(snapshotRoot, snapshotName);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Delete a snapshot of a snapshottable directory.
-   * 
-   * @param snapshotRoot The snapshottable directory that the 
-   *                    to-be-deleted snapshot belongs to
-   * @param snapshotName The name of the to-be-deleted snapshot
-   * @throws IOException
-   * @see ClientProtocol#deleteSnapshot(String, String)
-   */
-  public void deleteSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("deleteSnapshot", traceSampler);
-    try {
-      namenode.deleteSnapshot(snapshotRoot, snapshotName);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {

<TRUNCATED>

[11/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
new file mode 100644
index 0000000..7cfc854
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -0,0 +1,3203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Proxy;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.security.GeneralSecurityException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.net.SocketFactory;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoCodec;
+import org.apache.hadoop.crypto.CryptoInputStream;
+import org.apache.hadoop.crypto.CryptoOutputStream;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.HdfsBlockLocation;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.VolumeId;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.AclException;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcInvocationHandler;
+import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.tracing.SpanReceiverHost;
+import org.apache.hadoop.tracing.TraceUtils;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DataChecksum.Type;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.SamplerBuilder;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.net.InetAddresses;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/********************************************************
+ * DFSClient can connect to a Hadoop Filesystem and 
+ * perform basic file tasks.  It uses the ClientProtocol
+ * to communicate with a NameNode daemon, and connects 
+ * directly to DataNodes to read/write block data.
+ *
+ * Hadoop DFS users should obtain an instance of 
+ * DistributedFileSystem, which uses DFSClient to handle
+ * filesystem tasks.
+ *
+ ********************************************************/
+@InterfaceAudience.Private
+public class DFSClient implements java.io.Closeable, RemotePeerFactory,
+    DataEncryptionKeyFactory {
+  public static final Logger LOG = LoggerFactory.getLogger(DFSClient.class);
+  public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
+
+  private final Configuration conf;
+  private final DfsClientConf dfsClientConf;
+  final ClientProtocol namenode;
+  /* The service used for delegation tokens */
+  private Text dtService;
+
+  final UserGroupInformation ugi;
+  volatile boolean clientRunning = true;
+  volatile long lastLeaseRenewal;
+  private volatile FsServerDefaults serverDefaults;
+  private volatile long serverDefaultsLastUpdate;
+  final String clientName;
+  final SocketFactory socketFactory;
+  final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
+  final FileSystem.Statistics stats;
+  private final String authority;
+  private final Random r = new Random();
+  private SocketAddress[] localInterfaceAddrs;
+  private DataEncryptionKey encryptionKey;
+  final SaslDataTransferClient saslClient;
+  private final CachingStrategy defaultReadCachingStrategy;
+  private final CachingStrategy defaultWriteCachingStrategy;
+  private final ClientContext clientContext;
+
+  private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
+      new DFSHedgedReadMetrics();
+  private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
+  private final Sampler<?> traceSampler;
+  private final int smallBufferSize;
+
+  public DfsClientConf getConf() {
+    return dfsClientConf;
+  }
+
+  Configuration getConfiguration() {
+    return conf;
+  }
+
+  /**
+   * A map from file names to {@link DFSOutputStream} objects
+   * that are currently being written by this client.
+   * Note that a file can only be written by a single client.
+   */
+  private final Map<Long, DFSOutputStream> filesBeingWritten
+      = new HashMap<Long, DFSOutputStream>();
+
+  /**
+   * Same as this(NameNode.getNNAddress(conf), conf);
+   * @see #DFSClient(InetSocketAddress, Configuration)
+   * @deprecated Deprecated at 0.21
+   */
+  @Deprecated
+  public DFSClient(Configuration conf) throws IOException {
+    this(DFSUtilClient.getNNAddress(conf), conf);
+  }
+  
+  public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
+    this(DFSUtilClient.getNNUri(address), conf);
+  }
+
+  /**
+   * Same as this(nameNodeUri, conf, null);
+   * @see #DFSClient(URI, Configuration, FileSystem.Statistics)
+   */
+  public DFSClient(URI nameNodeUri, Configuration conf
+      ) throws IOException {
+    this(nameNodeUri, conf, null);
+  }
+
+  /**
+   * Same as this(nameNodeUri, null, conf, stats);
+   * @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics) 
+   */
+  public DFSClient(URI nameNodeUri, Configuration conf,
+                   FileSystem.Statistics stats)
+    throws IOException {
+    this(nameNodeUri, null, conf, stats);
+  }
+  
+  /** 
+   * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
+   * If HA is enabled and a positive value is set for
+   * {@link HdfsClientConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY}
+   * in the configuration, the DFSClient will use
+   * {@link LossyRetryInvocationHandler} as its RetryInvocationHandler.
+   * Otherwise one of nameNodeUri or rpcNamenode must be null.
+   */
+  @VisibleForTesting
+  public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
+      Configuration conf, FileSystem.Statistics stats)
+    throws IOException {
+    SpanReceiverHost.get(conf, HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
+    traceSampler = new SamplerBuilder(TraceUtils.
+        wrapHadoopConf(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf))
+        .build();
+    // Copy only the required DFSClient configuration
+    this.dfsClientConf = new DfsClientConf(conf);
+    this.conf = conf;
+    this.stats = stats;
+    this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
+    this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
+    this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
+
+    this.ugi = UserGroupInformation.getCurrentUser();
+    
+    this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
+    this.clientName = "DFSClient_" + dfsClientConf.getTaskId() + "_" + 
+        ThreadLocalRandom.current().nextInt()  + "_" +
+        Thread.currentThread().getId();
+    int numResponseToDrop = conf.getInt(
+        HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
+        HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
+    ProxyAndInfo<ClientProtocol> proxyInfo = null;
+    AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
+
+    if (numResponseToDrop > 0) {
+      // This case is used for testing.
+      LOG.warn(HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
+          + " is set to " + numResponseToDrop
+          + ", this hacked client will proactively drop responses");
+      proxyInfo = NameNodeProxiesClient.createProxyWithLossyRetryHandler(conf,
+          nameNodeUri, ClientProtocol.class, numResponseToDrop,
+          nnFallbackToSimpleAuth);
+    }
+    
+    if (proxyInfo != null) {
+      this.dtService = proxyInfo.getDelegationTokenService();
+      this.namenode = proxyInfo.getProxy();
+    } else if (rpcNamenode != null) {
+      // This case is used for testing.
+      Preconditions.checkArgument(nameNodeUri == null);
+      this.namenode = rpcNamenode;
+      dtService = null;
+    } else {
+      Preconditions.checkArgument(nameNodeUri != null,
+          "null URI");
+      proxyInfo = NameNodeProxiesClient.createProxyWithClientProtocol(conf,
+          nameNodeUri, nnFallbackToSimpleAuth);
+      this.dtService = proxyInfo.getDelegationTokenService();
+      this.namenode = proxyInfo.getProxy();
+    }
+
+    String localInterfaces[] =
+      conf.getTrimmedStrings(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
+    localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
+    if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
+      LOG.debug("Using local interfaces [" +
+      Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
+      Joiner.on(',').join(localInterfaceAddrs) + "]");
+    }
+    
+    Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
+        null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
+    Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
+        null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
+    Boolean writeDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
+        null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
+    this.defaultReadCachingStrategy =
+        new CachingStrategy(readDropBehind, readahead);
+    this.defaultWriteCachingStrategy =
+        new CachingStrategy(writeDropBehind, readahead);
+    this.clientContext = ClientContext.get(
+        conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
+        dfsClientConf);
+
+    if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
+      this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
+    }
+    this.saslClient = new SaslDataTransferClient(
+      conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
+      TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
+  }
+  
+  /**
+   * Return the socket addresses to use with each configured
+   * local interface. Local interfaces may be specified by IP
+   * address, IP address range using CIDR notation, interface
+   * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
+   * The socket addresses consist of the IPs for the interfaces
+   * and the ephemeral port (port 0). If an IP, IP range, or
+   * interface name matches an interface with sub-interfaces
+   * only the IP of the interface is used. Sub-interfaces can
+   * be used by specifying them explicitly (by IP or name).
+   * 
+   * @return SocketAddresses for the configured local interfaces,
+   *    or an empty array if none are configured
+   * @throws UnknownHostException if a given interface name is invalid
+   */
+  private static SocketAddress[] getLocalInterfaceAddrs(
+      String interfaceNames[]) throws UnknownHostException {
+    List<SocketAddress> localAddrs = new ArrayList<SocketAddress>();
+    for (String interfaceName : interfaceNames) {
+      if (InetAddresses.isInetAddress(interfaceName)) {
+        localAddrs.add(new InetSocketAddress(interfaceName, 0));
+      } else if (NetUtils.isValidSubnet(interfaceName)) {
+        for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
+          localAddrs.add(new InetSocketAddress(addr, 0));
+        }
+      } else {
+        for (String ip : DNS.getIPs(interfaceName, false)) {
+          localAddrs.add(new InetSocketAddress(ip, 0));
+        }
+      }
+    }
+    return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
+  }
+
+  /**
+   * Select one of the configured local interfaces at random. We use a random
+   * interface because other policies like round-robin are less effective
+   * given that we cache connections to datanodes.
+   *
+   * @return one of the local interface addresses at random, or null if no
+   *    local interfaces are configured
+   */
+  SocketAddress getRandomLocalInterfaceAddr() {
+    if (localInterfaceAddrs.length == 0) {
+      return null;
+    }
+    final int idx = r.nextInt(localInterfaceAddrs.length);
+    final SocketAddress addr = localInterfaceAddrs[idx];
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using local interface " + addr);
+    }
+    return addr;
+  }
+
+  /**
+   * Return the timeout that clients should use when writing to datanodes.
+   * @param numNodes the number of nodes in the pipeline.
+   */
+  int getDatanodeWriteTimeout(int numNodes) {
+    final int t = dfsClientConf.getDatanodeSocketWriteTimeout();
+    return t > 0? t + HdfsConstants.WRITE_TIMEOUT_EXTENSION*numNodes: 0;
+  }
+
+  int getDatanodeReadTimeout(int numNodes) {
+    final int t = dfsClientConf.getSocketTimeout();
+    return t > 0? HdfsConstants.READ_TIMEOUT_EXTENSION*numNodes + t: 0;
+  }
+  
+  @VisibleForTesting
+  public String getClientName() {
+    return clientName;
+  }
+
+  void checkOpen() throws IOException {
+    if (!clientRunning) {
+      IOException result = new IOException("Filesystem closed");
+      throw result;
+    }
+  }
+
+  /** Return the lease renewer instance. The renewer thread won't start
+   *  until the first output stream is created. The same instance will
+   *  be returned until all output streams are closed.
+   */
+  public LeaseRenewer getLeaseRenewer() throws IOException {
+      return LeaseRenewer.getInstance(authority, ugi, this);
+  }
+
+  /** Get a lease and start automatic renewal */
+  private void beginFileLease(final long inodeId, final DFSOutputStream out)
+      throws IOException {
+    getLeaseRenewer().put(inodeId, out, this);
+  }
+
+  /** Stop renewal of lease for the file. */
+  void endFileLease(final long inodeId) throws IOException {
+    getLeaseRenewer().closeFile(inodeId, this);
+  }
+    
+
+  /** Put a file. Only called from LeaseRenewer, where proper locking is
+   *  enforced to consistently update its local dfsclients array and 
+   *  client's filesBeingWritten map.
+   */
+  public void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
+    synchronized(filesBeingWritten) {
+      filesBeingWritten.put(inodeId, out);
+      // update the last lease renewal time only when there was no
+      // writes. once there is one write stream open, the lease renewer
+      // thread keeps it updated well with in anyone's expiration time.
+      if (lastLeaseRenewal == 0) {
+        updateLastLeaseRenewal();
+      }
+    }
+  }
+
+  /** Remove a file. Only called from LeaseRenewer. */
+  public void removeFileBeingWritten(final long inodeId) {
+    synchronized(filesBeingWritten) {
+      filesBeingWritten.remove(inodeId);
+      if (filesBeingWritten.isEmpty()) {
+        lastLeaseRenewal = 0;
+      }
+    }
+  }
+
+  /** Is file-being-written map empty? */
+  public boolean isFilesBeingWrittenEmpty() {
+    synchronized(filesBeingWritten) {
+      return filesBeingWritten.isEmpty();
+    }
+  }
+  
+  /** @return true if the client is running */
+  public boolean isClientRunning() {
+    return clientRunning;
+  }
+
+  long getLastLeaseRenewal() {
+    return lastLeaseRenewal;
+  }
+
+  void updateLastLeaseRenewal() {
+    synchronized(filesBeingWritten) {
+      if (filesBeingWritten.isEmpty()) {
+        return;
+      }
+      lastLeaseRenewal = Time.monotonicNow();
+    }
+  }
+
+  /**
+   * Renew leases.
+   * @return true if lease was renewed. May return false if this
+   * client has been closed or has no files open.
+   **/
+  public boolean renewLease() throws IOException {
+    if (clientRunning && !isFilesBeingWrittenEmpty()) {
+      try {
+        namenode.renewLease(clientName);
+        updateLastLeaseRenewal();
+        return true;
+      } catch (IOException e) {
+        // Abort if the lease has already expired. 
+        final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
+        if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
+          LOG.warn("Failed to renew lease for " + clientName + " for "
+              + (elapsed/1000) + " seconds (>= hard-limit ="
+              + (HdfsConstants.LEASE_HARDLIMIT_PERIOD / 1000) + " seconds.) "
+              + "Closing all files being written ...", e);
+          closeAllFilesBeingWritten(true);
+        } else {
+          // Let the lease renewer handle it and retry.
+          throw e;
+        }
+      }
+    }
+    return false;
+  }
+  
+  /**
+   * Close connections the Namenode.
+   */
+  void closeConnectionToNamenode() {
+    RPC.stopProxy(namenode);
+  }
+
+  /** Close/abort all files being written. */
+  public void closeAllFilesBeingWritten(final boolean abort) {
+    for(;;) {
+      final long inodeId;
+      final DFSOutputStream out;
+      synchronized(filesBeingWritten) {
+        if (filesBeingWritten.isEmpty()) {
+          return;
+        }
+        inodeId = filesBeingWritten.keySet().iterator().next();
+        out = filesBeingWritten.remove(inodeId);
+      }
+      if (out != null) {
+        try {
+          if (abort) {
+            out.abort();
+          } else {
+            out.close();
+          }
+        } catch(IOException ie) {
+          LOG.error("Failed to " + (abort ? "abort" : "close") + " file: "
+              + out.getSrc() + " with inode: " + inodeId, ie);
+        }
+      }
+    }
+  }
+
+  /**
+   * Close the file system, abandoning all of the leases and files being
+   * created and close connections to the namenode.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    if(clientRunning) {
+      closeAllFilesBeingWritten(false);
+      clientRunning = false;
+      getLeaseRenewer().closeClient(this);
+      // close connections to the namenode
+      closeConnectionToNamenode();
+    }
+  }
+
+  /**
+   * Close all open streams, abandoning all of the leases and files being
+   * created.
+   * @param abort whether streams should be gracefully closed
+   */
+  public void closeOutputStreams(boolean abort) {
+    if (clientRunning) {
+      closeAllFilesBeingWritten(abort);
+    }
+  }
+
+  /**
+   * @see ClientProtocol#getPreferredBlockSize(String)
+   */
+  public long getBlockSize(String f) throws IOException {
+    TraceScope scope = getPathTraceScope("getBlockSize", f);
+    try {
+      return namenode.getPreferredBlockSize(f);
+    } catch (IOException ie) {
+      LOG.warn("Problem getting block size", ie);
+      throw ie;
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Get server default values for a number of configuration params.
+   * @see ClientProtocol#getServerDefaults()
+   */
+  public FsServerDefaults getServerDefaults() throws IOException {
+    long now = Time.monotonicNow();
+    if ((serverDefaults == null) ||
+        (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
+      serverDefaults = namenode.getServerDefaults();
+      serverDefaultsLastUpdate = now;
+    }
+    assert serverDefaults != null;
+    return serverDefaults;
+  }
+  
+  /**
+   * Get a canonical token service name for this client's tokens.  Null should
+   * be returned if the client is not using tokens.
+   * @return the token service for the client
+   */
+  @InterfaceAudience.LimitedPrivate( { "HDFS" }) 
+  public String getCanonicalServiceName() {
+    return (dtService != null) ? dtService.toString() : null;
+  }
+  
+  /**
+   * @see ClientProtocol#getDelegationToken(Text)
+   */
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+      throws IOException {
+    assert dtService != null;
+    TraceScope scope = Trace.startSpan("getDelegationToken", traceSampler);
+    try {
+      Token<DelegationTokenIdentifier> token =
+        namenode.getDelegationToken(renewer);
+      if (token != null) {
+        token.setService(this.dtService);
+        LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
+      } else {
+        LOG.info("Cannot get delegation token from " + renewer);
+      }
+      return token;
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Renew a delegation token
+   * @param token the token to renew
+   * @return the new expiration time
+   * @throws InvalidToken
+   * @throws IOException
+   * @deprecated Use Token.renew instead.
+   */
+  @Deprecated
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws InvalidToken, IOException {
+    LOG.info("Renewing " + DelegationTokenIdentifier.stringifyToken(token));
+    try {
+      return token.renew(conf);
+    } catch (InterruptedException ie) {                                       
+      throw new RuntimeException("caught interrupted", ie);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(InvalidToken.class,
+                                     AccessControlException.class);
+    }
+  }
+  
+  /**
+   * Cancel a delegation token
+   * @param token the token to cancel
+   * @throws InvalidToken
+   * @throws IOException
+   * @deprecated Use Token.cancel instead.
+   */
+  @Deprecated
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws InvalidToken, IOException {
+    LOG.info("Cancelling " + DelegationTokenIdentifier.stringifyToken(token));
+    try {
+      token.cancel(conf);
+     } catch (InterruptedException ie) {                                       
+      throw new RuntimeException("caught interrupted", ie);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(InvalidToken.class,
+                                     AccessControlException.class);
+    }
+  }
+  
+  @InterfaceAudience.Private
+  public static class Renewer extends TokenRenewer {
+    
+    static {
+      //Ensure that HDFS Configuration files are loaded before trying to use
+      // the renewer.
+      HdfsConfigurationLoader.init();
+    }
+    
+    @Override
+    public boolean handleKind(Text kind) {
+      return DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public long renew(Token<?> token, Configuration conf) throws IOException {
+      Token<DelegationTokenIdentifier> delToken = 
+        (Token<DelegationTokenIdentifier>) token;
+      ClientProtocol nn = getNNProxy(delToken, conf);
+      try {
+        return nn.renewDelegationToken(delToken);
+      } catch (RemoteException re) {
+        throw re.unwrapRemoteException(InvalidToken.class, 
+                                       AccessControlException.class);
+      }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void cancel(Token<?> token, Configuration conf) throws IOException {
+      Token<DelegationTokenIdentifier> delToken = 
+          (Token<DelegationTokenIdentifier>) token;
+      LOG.info("Cancelling " + 
+               DelegationTokenIdentifier.stringifyToken(delToken));
+      ClientProtocol nn = getNNProxy(delToken, conf);
+      try {
+        nn.cancelDelegationToken(delToken);
+      } catch (RemoteException re) {
+        throw re.unwrapRemoteException(InvalidToken.class,
+            AccessControlException.class);
+      }
+    }
+    
+    private static ClientProtocol getNNProxy(
+        Token<DelegationTokenIdentifier> token, Configuration conf)
+        throws IOException {
+      URI uri = HAUtilClient.getServiceUriFromToken(
+          HdfsConstants.HDFS_URI_SCHEME, token);
+      if (HAUtilClient.isTokenForLogicalUri(token) &&
+          !HAUtilClient.isLogicalUri(conf, uri)) {
+        // If the token is for a logical nameservice, but the configuration
+        // we have disagrees about that, we can't actually renew it.
+        // This can be the case in MR, for example, if the RM doesn't
+        // have all of the HA clusters configured in its configuration.
+        throw new IOException("Unable to map logical nameservice URI '" +
+            uri + "' to a NameNode. Local configuration does not have " +
+            "a failover proxy provider configured.");
+      }
+      
+      ProxyAndInfo<ClientProtocol> info =
+        NameNodeProxiesClient.createProxyWithClientProtocol(conf, uri, null);
+      assert info.getDelegationTokenService().equals(token.getService()) :
+        "Returned service '" + info.getDelegationTokenService().toString() +
+        "' doesn't match expected service '" +
+        token.getService().toString() + "'";
+        
+      return info.getProxy();
+    }
+
+    @Override
+    public boolean isManaged(Token<?> token) throws IOException {
+      return true;
+    }
+    
+  }
+
+  /**
+   * Report corrupt blocks that were discovered by the client.
+   * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
+   */
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    namenode.reportBadBlocks(blocks);
+  }
+  
+  public LocatedBlocks getLocatedBlocks(String src, long start)
+      throws IOException {
+    return getLocatedBlocks(src, start, dfsClientConf.getPrefetchSize());
+  }
+
+  /*
+   * This is just a wrapper around callGetBlockLocations, but non-static so that
+   * we can stub it out for tests.
+   */
+  @VisibleForTesting
+  public LocatedBlocks getLocatedBlocks(String src, long start, long length)
+      throws IOException {
+    TraceScope scope = getPathTraceScope("getBlockLocations", src);
+    try {
+      return callGetBlockLocations(namenode, src, start, length);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * @see ClientProtocol#getBlockLocations(String, long, long)
+   */
+  static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
+      String src, long start, long length) 
+      throws IOException {
+    try {
+      return namenode.getBlockLocations(src, start, length);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
+    }
+  }
+
+  /**
+   * Recover a file's lease
+   * @param src a file's path
+   * @return true if the file is already closed
+   * @throws IOException
+   */
+  boolean recoverLease(String src) throws IOException {
+    checkOpen();
+
+    TraceScope scope = getPathTraceScope("recoverLease", src);
+    try {
+      return namenode.recoverLease(src, clientName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(FileNotFoundException.class,
+                                     AccessControlException.class,
+                                     UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Get block location info about file
+   * 
+   * getBlockLocations() returns a list of hostnames that store 
+   * data for a specific file region.  It returns a set of hostnames
+   * for every block within the indicated region.
+   *
+   * This function is very useful when writing code that considers
+   * data-placement when performing operations.  For example, the
+   * MapReduce system tries to schedule tasks on the same machines
+   * as the data-block the task processes. 
+   */
+  public BlockLocation[] getBlockLocations(String src, long start, 
+        long length) throws IOException, UnresolvedLinkException {
+    TraceScope scope = getPathTraceScope("getBlockLocations", src);
+    try {
+      LocatedBlocks blocks = getLocatedBlocks(src, start, length);
+      BlockLocation[] locations =  DFSUtilClient.locatedBlocks2Locations(blocks);
+      HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
+      for (int i = 0; i < locations.length; i++) {
+        hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
+      }
+      return hdfsLocations;
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Get block location information about a list of {@link HdfsBlockLocation}.
+   * Used by {@link DistributedFileSystem#getFileBlockStorageLocations(List)} to
+   * get {@link BlockStorageLocation}s for blocks returned by
+   * {@link DistributedFileSystem#getFileBlockLocations(org.apache.hadoop.fs.FileStatus, long, long)}
+   * .
+   * 
+   * This is done by making a round of RPCs to the associated datanodes, asking
+   * the volume of each block replica. The returned array of
+   * {@link BlockStorageLocation} expose this information as a
+   * {@link VolumeId}.
+   * 
+   * @param blockLocations
+   *          target blocks on which to query volume location information
+   * @return volumeBlockLocations original block array augmented with additional
+   *         volume location information for each replica.
+   */
+  public BlockStorageLocation[] getBlockStorageLocations(
+      List<BlockLocation> blockLocations) throws IOException,
+      UnsupportedOperationException, InvalidBlockTokenException {
+    if (!getConf().isHdfsBlocksMetadataEnabled()) {
+      throw new UnsupportedOperationException("Datanode-side support for " +
+          "getVolumeBlockLocations() must also be enabled in the client " +
+          "configuration.");
+    }
+    // Downcast blockLocations and fetch out required LocatedBlock(s)
+    List<LocatedBlock> blocks = new ArrayList<LocatedBlock>();
+    for (BlockLocation loc : blockLocations) {
+      if (!(loc instanceof HdfsBlockLocation)) {
+        throw new ClassCastException("DFSClient#getVolumeBlockLocations " +
+            "expected to be passed HdfsBlockLocations");
+      }
+      HdfsBlockLocation hdfsLoc = (HdfsBlockLocation) loc;
+      blocks.add(hdfsLoc.getLocatedBlock());
+    }
+    
+    // Re-group the LocatedBlocks to be grouped by datanodes, with the values
+    // a list of the LocatedBlocks on the datanode.
+    Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks = 
+        new LinkedHashMap<DatanodeInfo, List<LocatedBlock>>();
+    for (LocatedBlock b : blocks) {
+      for (DatanodeInfo info : b.getLocations()) {
+        if (!datanodeBlocks.containsKey(info)) {
+          datanodeBlocks.put(info, new ArrayList<LocatedBlock>());
+        }
+        List<LocatedBlock> l = datanodeBlocks.get(info);
+        l.add(b);
+      }
+    }
+        
+    // Make RPCs to the datanodes to get volume locations for its replicas
+    TraceScope scope =
+      Trace.startSpan("getBlockStorageLocations", traceSampler);
+    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas;
+    try {
+      metadatas = BlockStorageLocationUtil.
+          queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
+              getConf().getFileBlockStorageLocationsNumThreads(),
+              getConf().getFileBlockStorageLocationsTimeoutMs(),
+              getConf().isConnectToDnViaHostname());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("metadata returned: "
+            + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
+      }
+    } finally {
+      scope.close();
+    }
+    
+    // Regroup the returned VolumeId metadata to again be grouped by
+    // LocatedBlock rather than by datanode
+    Map<LocatedBlock, List<VolumeId>> blockVolumeIds = BlockStorageLocationUtil
+        .associateVolumeIdsWithBlocks(blocks, metadatas);
+    
+    // Combine original BlockLocations with new VolumeId information
+    BlockStorageLocation[] volumeBlockLocations = BlockStorageLocationUtil
+        .convertToVolumeBlockLocations(blocks, blockVolumeIds);
+
+    return volumeBlockLocations;
+  }
+
+  /**
+   * Decrypts a EDEK by consulting the KeyProvider.
+   */
+  private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
+      feInfo) throws IOException {
+    TraceScope scope = Trace.startSpan("decryptEDEK", traceSampler);
+    try {
+      KeyProvider provider = getKeyProvider();
+      if (provider == null) {
+        throw new IOException("No KeyProvider is configured, cannot access" +
+            " an encrypted file");
+      }
+      EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
+          feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
+          feInfo.getEncryptedDataEncryptionKey());
+      try {
+        KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
+            .createKeyProviderCryptoExtension(provider);
+        return cryptoProvider.decryptEncryptedKey(ekv);
+      } catch (GeneralSecurityException e) {
+        throw new IOException(e);
+      }
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Obtain the crypto protocol version from the provided FileEncryptionInfo,
+   * checking to see if this version is supported by.
+   *
+   * @param feInfo FileEncryptionInfo
+   * @return CryptoProtocolVersion from the feInfo
+   * @throws IOException if the protocol version is unsupported.
+   */
+  private static CryptoProtocolVersion getCryptoProtocolVersion
+      (FileEncryptionInfo feInfo) throws IOException {
+    final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
+    if (!CryptoProtocolVersion.supports(version)) {
+      throw new IOException("Client does not support specified " +
+          "CryptoProtocolVersion " + version.getDescription() + " version " +
+          "number" + version.getVersion());
+    }
+    return version;
+  }
+
+  /**
+   * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
+   * and the available CryptoCodecs configured in the Configuration.
+   *
+   * @param conf   Configuration
+   * @param feInfo FileEncryptionInfo
+   * @return CryptoCodec
+   * @throws IOException if no suitable CryptoCodec for the CipherSuite is
+   *                     available.
+   */
+  private static CryptoCodec getCryptoCodec(Configuration conf,
+      FileEncryptionInfo feInfo) throws IOException {
+    final CipherSuite suite = feInfo.getCipherSuite();
+    if (suite.equals(CipherSuite.UNKNOWN)) {
+      throw new IOException("NameNode specified unknown CipherSuite with ID "
+          + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
+    }
+    final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
+    if (codec == null) {
+      throw new UnknownCipherSuiteException(
+          "No configuration found for the cipher suite "
+          + suite.getConfigSuffix() + " prefixed with "
+          + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+          + ". Please see the example configuration "
+          + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
+          + "at core-default.xml for details.");
+    }
+    return codec;
+  }
+
+  /**
+   * Wraps the stream in a CryptoInputStream if the underlying file is
+   * encrypted.
+   */
+  public HdfsDataInputStream createWrappedInputStream(DFSInputStream dfsis)
+      throws IOException {
+    final FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
+    if (feInfo != null) {
+      // File is encrypted, wrap the stream in a crypto stream.
+      // Currently only one version, so no special logic based on the version #
+      getCryptoProtocolVersion(feInfo);
+      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+      final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+      final CryptoInputStream cryptoIn =
+          new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
+              feInfo.getIV());
+      return new HdfsDataInputStream(cryptoIn);
+    } else {
+      // No FileEncryptionInfo so no encryption.
+      return new HdfsDataInputStream(dfsis);
+    }
+  }
+
+  /**
+   * Wraps the stream in a CryptoOutputStream if the underlying file is
+   * encrypted.
+   */
+  public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
+      FileSystem.Statistics statistics) throws IOException {
+    return createWrappedOutputStream(dfsos, statistics, 0);
+  }
+
+  /**
+   * Wraps the stream in a CryptoOutputStream if the underlying file is
+   * encrypted.
+   */
+  public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
+      FileSystem.Statistics statistics, long startPos) throws IOException {
+    final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
+    if (feInfo != null) {
+      // File is encrypted, wrap the stream in a crypto stream.
+      // Currently only one version, so no special logic based on the version #
+      getCryptoProtocolVersion(feInfo);
+      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+      KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+      final CryptoOutputStream cryptoOut =
+          new CryptoOutputStream(dfsos, codec,
+              decrypted.getMaterial(), feInfo.getIV(), startPos);
+      return new HdfsDataOutputStream(cryptoOut, statistics, startPos);
+    } else {
+      // No FileEncryptionInfo present so no encryption.
+      return new HdfsDataOutputStream(dfsos, statistics, startPos);
+    }
+  }
+
+  public DFSInputStream open(String src) 
+      throws IOException, UnresolvedLinkException {
+    return open(src, dfsClientConf.getIoBufferSize(), true, null);
+  }
+
+  /**
+   * Create an input stream that obtains a nodelist from the
+   * namenode, and then reads from all the right places.  Creates
+   * inner subclass of InputStream that does the right out-of-band
+   * work.
+   * @deprecated Use {@link #open(String, int, boolean)} instead.
+   */
+  @Deprecated
+  public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
+                             FileSystem.Statistics stats)
+      throws IOException, UnresolvedLinkException {
+    return open(src, buffersize, verifyChecksum);
+  }
+  
+
+  /**
+   * Create an input stream that obtains a nodelist from the
+   * namenode, and then reads from all the right places.  Creates
+   * inner subclass of InputStream that does the right out-of-band
+   * work.
+   */
+  public DFSInputStream open(String src, int buffersize, boolean verifyChecksum)
+      throws IOException, UnresolvedLinkException {
+    checkOpen();
+    //    Get block info from namenode
+    TraceScope scope = getPathTraceScope("newDFSInputStream", src);
+    try {
+      return new DFSInputStream(this, src, verifyChecksum, null);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Get the namenode associated with this DFSClient object
+   * @return the namenode associated with this DFSClient object
+   */
+  public ClientProtocol getNamenode() {
+    return namenode;
+  }
+  
+  /**
+   * Call {@link #create(String, boolean, short, long, Progressable)} with
+   * default <code>replication</code> and <code>blockSize<code> and null <code>
+   * progress</code>.
+   */
+  public OutputStream create(String src, boolean overwrite) 
+      throws IOException {
+    return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+        dfsClientConf.getDefaultBlockSize(), null);
+  }
+    
+  /**
+   * Call {@link #create(String, boolean, short, long, Progressable)} with
+   * default <code>replication</code> and <code>blockSize<code>.
+   */
+  public OutputStream create(String src, 
+                             boolean overwrite,
+                             Progressable progress) throws IOException {
+    return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+        dfsClientConf.getDefaultBlockSize(), progress);
+  }
+    
+  /**
+   * Call {@link #create(String, boolean, short, long, Progressable)} with
+   * null <code>progress</code>.
+   */
+  public OutputStream create(String src, 
+                             boolean overwrite, 
+                             short replication,
+                             long blockSize) throws IOException {
+    return create(src, overwrite, replication, blockSize, null);
+  }
+
+  /**
+   * Call {@link #create(String, boolean, short, long, Progressable, int)}
+   * with default bufferSize.
+   */
+  public OutputStream create(String src, boolean overwrite, short replication,
+      long blockSize, Progressable progress) throws IOException {
+    return create(src, overwrite, replication, blockSize, progress,
+        dfsClientConf.getIoBufferSize());
+  }
+
+  /**
+   * Call {@link #create(String, FsPermission, EnumSet, short, long, 
+   * Progressable, int, ChecksumOpt)} with default <code>permission</code>
+   * {@link FsPermission#getFileDefault()}.
+   * 
+   * @param src File name
+   * @param overwrite overwrite an existing file if true
+   * @param replication replication factor for the file
+   * @param blockSize maximum block size
+   * @param progress interface for reporting client progress
+   * @param buffersize underlying buffersize
+   * 
+   * @return output stream
+   */
+  public OutputStream create(String src,
+                             boolean overwrite,
+                             short replication,
+                             long blockSize,
+                             Progressable progress,
+                             int buffersize)
+      throws IOException {
+    return create(src, FsPermission.getFileDefault(),
+        overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+            : EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
+        buffersize, null);
+  }
+
+  /**
+   * Call {@link #create(String, FsPermission, EnumSet, boolean, short, 
+   * long, Progressable, int, ChecksumOpt)} with <code>createParent</code>
+   *  set to true.
+   */
+  public DFSOutputStream create(String src, 
+                             FsPermission permission,
+                             EnumSet<CreateFlag> flag, 
+                             short replication,
+                             long blockSize,
+                             Progressable progress,
+                             int buffersize,
+                             ChecksumOpt checksumOpt)
+      throws IOException {
+    return create(src, permission, flag, true,
+        replication, blockSize, progress, buffersize, checksumOpt, null);
+  }
+
+  /**
+   * Create a new dfs file with the specified block replication 
+   * with write-progress reporting and return an output stream for writing
+   * into the file.  
+   * 
+   * @param src File name
+   * @param permission The permission of the directory being created.
+   *          If null, use default permission {@link FsPermission#getFileDefault()}
+   * @param flag indicates create a new file or create/overwrite an
+   *          existing file or append to an existing file
+   * @param createParent create missing parent directory if true
+   * @param replication block replication
+   * @param blockSize maximum block size
+   * @param progress interface for reporting client progress
+   * @param buffersize underlying buffer size 
+   * @param checksumOpt checksum options
+   * 
+   * @return output stream
+   *
+   * @see ClientProtocol#create for detailed description of exceptions thrown
+   */
+  public DFSOutputStream create(String src, 
+                             FsPermission permission,
+                             EnumSet<CreateFlag> flag, 
+                             boolean createParent,
+                             short replication,
+                             long blockSize,
+                             Progressable progress,
+                             int buffersize,
+                             ChecksumOpt checksumOpt) throws IOException {
+    return create(src, permission, flag, createParent, replication, blockSize, 
+        progress, buffersize, checksumOpt, null);
+  }
+
+  private FsPermission applyUMask(FsPermission permission) {
+    if (permission == null) {
+      permission = FsPermission.getFileDefault();
+    }
+    return permission.applyUMask(dfsClientConf.getUMask());
+  }
+
+  /**
+   * Same as {@link #create(String, FsPermission, EnumSet, boolean, short, long,
+   * Progressable, int, ChecksumOpt)} with the addition of favoredNodes that is
+   * a hint to where the namenode should place the file blocks.
+   * The favored nodes hint is not persisted in HDFS. Hence it may be honored
+   * at the creation time only. HDFS could move the blocks during balancing or
+   * replication, to move the blocks from favored nodes. A value of null means
+   * no favored nodes for this create
+   */
+  public DFSOutputStream create(String src, 
+                             FsPermission permission,
+                             EnumSet<CreateFlag> flag, 
+                             boolean createParent,
+                             short replication,
+                             long blockSize,
+                             Progressable progress,
+                             int buffersize,
+                             ChecksumOpt checksumOpt,
+                             InetSocketAddress[] favoredNodes) throws IOException {
+    checkOpen();
+    final FsPermission masked = applyUMask(permission);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug(src + ": masked=" + masked);
+    }
+    final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
+        src, masked, flag, createParent, replication, blockSize, progress,
+        buffersize, dfsClientConf.createChecksum(checksumOpt),
+        getFavoredNodesStr(favoredNodes));
+    beginFileLease(result.getFileId(), result);
+    return result;
+  }
+
+  private String[] getFavoredNodesStr(InetSocketAddress[] favoredNodes) {
+    String[] favoredNodeStrs = null;
+    if (favoredNodes != null) {
+      favoredNodeStrs = new String[favoredNodes.length];
+      for (int i = 0; i < favoredNodes.length; i++) {
+        favoredNodeStrs[i] = 
+            favoredNodes[i].getHostName() + ":" 
+                         + favoredNodes[i].getPort();
+      }
+    }
+    return favoredNodeStrs;
+  }
+  
+  /**
+   * Append to an existing file if {@link CreateFlag#APPEND} is present
+   */
+  private DFSOutputStream primitiveAppend(String src, EnumSet<CreateFlag> flag,
+      int buffersize, Progressable progress) throws IOException {
+    if (flag.contains(CreateFlag.APPEND)) {
+      HdfsFileStatus stat = getFileInfo(src);
+      if (stat == null) { // No file to append to
+        // New file needs to be created if create option is present
+        if (!flag.contains(CreateFlag.CREATE)) {
+          throw new FileNotFoundException("failed to append to non-existent file "
+              + src + " on client " + clientName);
+        }
+        return null;
+      }
+      return callAppend(src, buffersize, flag, progress, null);
+    }
+    return null;
+  }
+  
+  /**
+   * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
+   *  Progressable, int, ChecksumOpt)} except that the permission
+   *  is absolute (ie has already been masked with umask.
+   */
+  public DFSOutputStream primitiveCreate(String src, 
+                             FsPermission absPermission,
+                             EnumSet<CreateFlag> flag,
+                             boolean createParent,
+                             short replication,
+                             long blockSize,
+                             Progressable progress,
+                             int buffersize,
+                             ChecksumOpt checksumOpt)
+      throws IOException, UnresolvedLinkException {
+    checkOpen();
+    CreateFlag.validate(flag);
+    DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress);
+    if (result == null) {
+      DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt);
+      result = DFSOutputStream.newStreamForCreate(this, src, absPermission,
+          flag, createParent, replication, blockSize, progress, buffersize,
+          checksum, null);
+    }
+    beginFileLease(result.getFileId(), result);
+    return result;
+  }
+  
+  /**
+   * Creates a symbolic link.
+   * 
+   * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean) 
+   */
+  public void createSymlink(String target, String link, boolean createParent)
+      throws IOException {
+    TraceScope scope = getPathTraceScope("createSymlink", target);
+    try {
+      final FsPermission dirPerm = applyUMask(null);
+      namenode.createSymlink(target, link, dirPerm, createParent);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileAlreadyExistsException.class, 
+                                     FileNotFoundException.class,
+                                     ParentNotDirectoryException.class,
+                                     NSQuotaExceededException.class, 
+                                     DSQuotaExceededException.class,
+                                     QuotaByStorageTypeExceededException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Resolve the *first* symlink, if any, in the path.
+   * 
+   * @see ClientProtocol#getLinkTarget(String)
+   */
+  public String getLinkTarget(String path) throws IOException { 
+    checkOpen();
+    TraceScope scope = getPathTraceScope("getLinkTarget", path);
+    try {
+      return namenode.getLinkTarget(path);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /** Method to get stream returned by append call */
+  private DFSOutputStream callAppend(String src, int buffersize,
+      EnumSet<CreateFlag> flag, Progressable progress, String[] favoredNodes)
+      throws IOException {
+    CreateFlag.validateForAppend(flag);
+    try {
+      LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
+          new EnumSetWritable<>(flag, CreateFlag.class));
+      return DFSOutputStream.newStreamForAppend(this, src, flag, buffersize,
+          progress, blkWithStatus.getLastBlock(),
+          blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(null),
+          favoredNodes);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
+                                     DSQuotaExceededException.class,
+                                     QuotaByStorageTypeExceededException.class,
+                                     UnsupportedOperationException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    }
+  }
+  
+  /**
+   * Append to an existing HDFS file.  
+   * 
+   * @param src file name
+   * @param buffersize buffer size
+   * @param flag indicates whether to append data to a new block instead of
+   *             the last block
+   * @param progress for reporting write-progress; null is acceptable.
+   * @param statistics file system statistics; null is acceptable.
+   * @return an output stream for writing into the file
+   * 
+   * @see ClientProtocol#append(String, String, EnumSetWritable)
+   */
+  public HdfsDataOutputStream append(final String src, final int buffersize,
+      EnumSet<CreateFlag> flag, final Progressable progress,
+      final FileSystem.Statistics statistics) throws IOException {
+    final DFSOutputStream out = append(src, buffersize, flag, null, progress);
+    return createWrappedOutputStream(out, statistics, out.getInitialLen());
+  }
+
+  /**
+   * Append to an existing HDFS file.
+   * 
+   * @param src file name
+   * @param buffersize buffer size
+   * @param flag indicates whether to append data to a new block instead of the
+   *          last block
+   * @param progress for reporting write-progress; null is acceptable.
+   * @param statistics file system statistics; null is acceptable.
+   * @param favoredNodes FavoredNodes for new blocks
+   * @return an output stream for writing into the file
+   * @see ClientProtocol#append(String, String, EnumSetWritable)
+   */
+  public HdfsDataOutputStream append(final String src, final int buffersize,
+      EnumSet<CreateFlag> flag, final Progressable progress,
+      final FileSystem.Statistics statistics,
+      final InetSocketAddress[] favoredNodes) throws IOException {
+    final DFSOutputStream out = append(src, buffersize, flag,
+        getFavoredNodesStr(favoredNodes), progress);
+    return createWrappedOutputStream(out, statistics, out.getInitialLen());
+  }
+
+  private DFSOutputStream append(String src, int buffersize,
+      EnumSet<CreateFlag> flag, String[] favoredNodes, Progressable progress)
+      throws IOException {
+    checkOpen();
+    final DFSOutputStream result = callAppend(src, buffersize, flag, progress,
+        favoredNodes);
+    beginFileLease(result.getFileId(), result);
+    return result;
+  }
+
+  /**
+   * Set replication for an existing file.
+   * @param src file name
+   * @param replication replication to set the file to
+   * 
+   * @see ClientProtocol#setReplication(String, short)
+   */
+  public boolean setReplication(String src, short replication)
+      throws IOException {
+    TraceScope scope = getPathTraceScope("setReplication", src);
+    try {
+      return namenode.setReplication(src, replication);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
+                                     DSQuotaExceededException.class,
+                                     QuotaByStorageTypeExceededException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Set storage policy for an existing file/directory
+   * @param src file/directory name
+   * @param policyName name of the storage policy
+   */
+  public void setStoragePolicy(String src, String policyName)
+      throws IOException {
+    TraceScope scope = getPathTraceScope("setStoragePolicy", src);
+    try {
+      namenode.setStoragePolicy(src, policyName);
+    } catch (RemoteException e) {
+      throw e.unwrapRemoteException(AccessControlException.class,
+                                    FileNotFoundException.class,
+                                    SafeModeException.class,
+                                    NSQuotaExceededException.class,
+                                    UnresolvedPathException.class,
+                                    SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * @param path file/directory name
+   * @return Get the storage policy for specified path
+   */
+  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("getStoragePolicy", path);
+    try {
+      return namenode.getStoragePolicy(path);
+    } catch (RemoteException e) {
+      throw e.unwrapRemoteException(AccessControlException.class,
+                                    FileNotFoundException.class,
+                                    SafeModeException.class,
+                                    UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * @return All the existing storage policies
+   */
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler);
+    try {
+      return namenode.getStoragePolicies();
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Rename file or directory.
+   * @see ClientProtocol#rename(String, String)
+   * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
+   */
+  @Deprecated
+  public boolean rename(String src, String dst) throws IOException {
+    checkOpen();
+    TraceScope scope = getSrcDstTraceScope("rename", src, dst);
+    try {
+      return namenode.rename(src, dst);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     NSQuotaExceededException.class,
+                                     DSQuotaExceededException.class,
+                                     QuotaByStorageTypeExceededException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Move blocks from src to trg and delete src
+   * See {@link ClientProtocol#concat}.
+   */
+  public void concat(String trg, String [] srcs) throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("concat", traceSampler);
+    try {
+      namenode.concat(trg, srcs);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+  /**
+   * Rename file or directory.
+   * @see ClientProtocol#rename2(String, String, Options.Rename...)
+   */
+  public void rename(String src, String dst, Options.Rename... options)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = getSrcDstTraceScope("rename2", src, dst);
+    try {
+      namenode.rename2(src, dst, options);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     DSQuotaExceededException.class,
+                                     QuotaByStorageTypeExceededException.class,
+                                     FileAlreadyExistsException.class,
+                                     FileNotFoundException.class,
+                                     ParentNotDirectoryException.class,
+                                     SafeModeException.class,
+                                     NSQuotaExceededException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Truncate a file to an indicated size
+   * See {@link ClientProtocol#truncate}.
+   */
+  public boolean truncate(String src, long newLength) throws IOException {
+    checkOpen();
+    if (newLength < 0) {
+      throw new HadoopIllegalArgumentException(
+          "Cannot truncate to a negative file size: " + newLength + ".");
+    }
+    try {
+      return namenode.truncate(src, newLength, clientName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          UnresolvedPathException.class);
+    }
+  }
+
+  /**
+   * Delete file or directory.
+   * See {@link ClientProtocol#delete(String, boolean)}. 
+   */
+  @Deprecated
+  public boolean delete(String src) throws IOException {
+    checkOpen();
+    return delete(src, true);
+  }
+
+  /**
+   * delete file or directory.
+   * delete contents of the directory if non empty and recursive 
+   * set to true
+   *
+   * @see ClientProtocol#delete(String, boolean)
+   */
+  public boolean delete(String src, boolean recursive) throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("delete", src);
+    try {
+      return namenode.delete(src, recursive);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /** Implemented using getFileInfo(src)
+   */
+  public boolean exists(String src) throws IOException {
+    checkOpen();
+    return getFileInfo(src) != null;
+  }
+
+  /**
+   * Get a partial listing of the indicated directory
+   * No block locations need to be fetched
+   */
+  public DirectoryListing listPaths(String src,  byte[] startAfter)
+    throws IOException {
+    return listPaths(src, startAfter, false);
+  }
+  
+  /**
+   * Get a partial listing of the indicated directory
+   *
+   * Recommend to use HdfsFileStatus.EMPTY_NAME as startAfter
+   * if the application wants to fetch a listing starting from
+   * the first entry in the directory
+   *
+   * @see ClientProtocol#getListing(String, byte[], boolean)
+   */
+  public DirectoryListing listPaths(String src,  byte[] startAfter,
+      boolean needLocation) throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("listPaths", src);
+    try {
+      return namenode.getListing(src, startAfter, needLocation);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Get the file info for a specific file or directory.
+   * @param src The string representation of the path to the file
+   * @return object containing information regarding the file
+   *         or null if file not found
+   *         
+   * @see ClientProtocol#getFileInfo(String) for description of exceptions
+   */
+  public HdfsFileStatus getFileInfo(String src) throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("getFileInfo", src);
+    try {
+      return namenode.getFileInfo(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Close status of a file
+   * @return true if file is already closed
+   */
+  public boolean isFileClosed(String src) throws IOException{
+    checkOpen();
+    TraceScope scope = getPathTraceScope("isFileClosed", src);
+    try {
+      return namenode.isFileClosed(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Get the file info for a specific file or directory. If src
+   * refers to a symlink then the FileStatus of the link is returned.
+   * @param src path to a file or directory.
+   * 
+   * For description of exceptions thrown 
+   * @see ClientProtocol#getFileLinkInfo(String)
+   */
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("getFileLinkInfo", src);
+    try {
+      return namenode.getFileLinkInfo(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+   }
+  
+  @InterfaceAudience.Private
+  public void clearDataEncryptionKey() {
+    LOG.debug("Clearing encryption key");
+    synchronized (this) {
+      encryptionKey = null;
+    }
+  }
+  
+  /**
+   * @return true if data sent between this client and DNs should be encrypted,
+   *         false otherwise.
+   * @throws IOException in the event of error communicating with the NN
+   */
+  boolean shouldEncryptData() throws IOException {
+    FsServerDefaults d = getServerDefaults();
+    return d == null ? false : d.getEncryptDataTransfer();
+  }
+  
+  @Override
+  public DataEncryptionKey newDataEncryptionKey() throws IOException {
+    if (shouldEncryptData()) {
+      synchronized (this) {
+        if (encryptionKey == null ||
+            encryptionKey.expiryDate < Time.now()) {
+          LOG.debug("Getting new encryption token from NN");
+          encryptionKey = namenode.getDataEncryptionKey();
+        }
+        return encryptionKey;
+      }
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Get the checksum of the whole file of a range of the file. Note that the
+   * range always starts from the beginning of the file.
+   * @param src The file path
+   * @param length the length of the range, i.e., the range is [0, length]
+   * @return The checksum 
+   * @see DistributedFileSystem#getFileChecksum(Path)
+   */
+  public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
+      throws IOException {
+    checkOpen();
+    Preconditions.checkArgument(length >= 0);
+    //get block locations for the file range
+    LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0,
+        length);
+    if (null == blockLocations) {
+      throw new FileNotFoundException("File does not exist: " + src);
+    }
+    if (blockLocations.isUnderConstruction()) {
+      throw new IOException("Fail to get checksum, since file " + src
+          + " is under construction.");
+    }
+    List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
+    final DataOutputBuffer md5out = new DataOutputBuffer();
+    int bytesPerCRC = -1;
+    DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
+    long crcPerBlock = 0;
+    boolean refetchBlocks = false;
+    int lastRetriedIndex = -1;
+
+    // get block checksum for each block
+    long remaining = length;
+    if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
+      remaining = Math.min(length, blockLocations.getFileLength());
+    }
+    for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
+      if (refetchBlocks) {  // refetch to get fresh tokens
+        blockLocations = callGetBlockLocations(namenode, src, 0, length);
+        if (null == blockLocations) {
+          throw new FileNotFoundException("File does not exist: " + src);
+        }
+        if (blockLocations.isUnderConstruction()) {
+          throw new IOException("Fail to get checksum, since file " + src
+              + " is under construction.");
+        }
+        locatedblocks = blockLocations.getLocatedBlocks();
+        refetchBlocks = false;
+      }
+      LocatedBlock lb = locatedblocks.get(i);
+      final ExtendedBlock block = lb.getBlock();
+      if (remaining < block.getNumBytes()) {
+        block.setNumBytes(remaining);
+      }
+      remaining -= block.getNumBytes();
+      final DatanodeInfo[] datanodes = lb.getLocations();
+      
+      //try each datanode location of the block
+      final int timeout = 3000*datanodes.length + dfsClientConf.getSocketTimeout();
+      boolean done = false;
+      for(int j = 0; !done && j < datanodes.length; j++) {
+        DataOutputStream out = null;
+        DataInputStream in = null;
+        
+        try {
+          //connect to a datanode
+          IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
+          out = new DataOutputStream(new BufferedOutputStream(pair.out,
+              smallBufferSize));
+          in = new DataInputStream(pair.in);
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("write to " + datanodes[j] + ": "
+                + Op.BLOCK_CHECKSUM + ", block=" + block);
+          }
+          // get block MD5
+          new Sender(out).blockChecksum(block, lb.getBlockToken());
+
+          final BlockOpResponseProto reply =
+            BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+
+          String logInfo = "for block " + block + " from datanode " + datanodes[j];
+          DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
+          OpBlockChecksumResponseProto checksumData =
+            reply.getChecksumResponse();
+
+          //read byte-per-checksum
+          final int bpc = checksumData.getBytesPerCrc();
+          if (i == 0) { //first block
+            bytesPerCRC = bpc;
+          }
+          else if (bpc != bytesPerCRC) {
+            throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
+                + " but bytesPerCRC=" + bytesPerCRC);
+          }
+          
+          //read crc-per-block
+          final long cpb = checksumData.getCrcPerBlock();
+          if (locatedblocks.size() > 1 && i == 0) {
+            crcPerBlock = cpb;
+          }
+
+          //read md5
+          final MD5Hash md5 = new MD5Hash(
+              checksumData.getMd5().toByteArray());
+          md5.write(md5out);
+          
+          // read crc-type
+          final DataChecksum.Type ct;
+          if (checksumData.hasCrcType()) {
+            ct = PBHelperClient.convert(checksumData
+                .getCrcType());
+          } else {
+            LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
+                      "inferring checksum by reading first byte");
+            ct = inferChecksumTypeByReading(lb, datanodes[j]);
+          }
+
+          if (i == 0) { // first block
+            crcType = ct;
+          } else if (crcType != DataChecksum.Type.MIXED
+              && crcType != ct) {
+            // if crc types are mixed in a file
+            crcType = DataChecksum.Type.MIXED;
+          }
+
+          done = true;
+
+          if (LOG.isDebugEnabled()) {
+            if (i == 0) {
+              LOG.debug("set bytesPerCRC=" + bytesPerCRC
+                  + ", crcPerBlock=" + crcPerBlock);
+            }
+            LOG.debug("got reply from " + datanodes[j] + ": md5=" + md5);
+          }
+        } catch (InvalidBlockTokenException ibte) {
+          if (i > lastRetriedIndex) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
+                  + "for file " + src + " for block " + block
+                  + " from datanode " + datanodes[j]
+                  + ". Will retry the block once.");
+            }
+            lastRetriedIndex = i;
+            done = true; // actually it's not done; but we'll retry
+            i--; // repeat at i-th block
+            refetchBlocks = true;
+            break;
+          }
+        } catch (IOException ie) {
+          LOG.warn("src=" + src + ", datanodes["+j+"]=" + datanodes[j], ie);
+        } finally {
+          IOUtils.closeStream(in);
+          IOUtils.closeStream(out);
+        }
+      }
+
+      if (!done) {
+        throw new IOException("Fail to get block MD5 for " + block);
+      }
+    }
+
+    //compute file MD5
+    final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData()); 
+    switch (crcType) {
+      case CRC32:
+        return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
+            crcPerBlock, fileMD5);
+      case CRC32C:
+        return new MD5MD5CRC32CastagnoliFileChecksum(bytesPerCRC,
+            crcPerBlock, fileMD5);
+      default:
+        // If there is no block allocated for the file,
+        // return one with the magic entry that matches what previous
+        // hdfs versions return.
+        if (locatedblocks.size() == 0) {
+          return new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
+        }
+
+        // we should never get here since the validity was checked
+        // when getCrcType() was called above.
+        return null;
+    }
+  }
+
+  /**
+   * Connect to the given datanode's datantrasfer port, and return
+   * the resulting IOStreamPair. This includes encryption wrapping, etc.
+   */
+  private IOStreamPair connectToDN(DatanodeInfo dn, int timeout,
+      LocatedBlock lb) throws IOException {
+    boolean success = false;
+    Socket sock = null;
+    try {
+      sock = socketFactory.createSocket();
+      String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Connecting to datanode " + dnAddr);
+      }
+      NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
+      sock.setSoTimeout(timeout);
+  
+      OutputStream unbufOut = NetUtils.getOutputStream(sock);
+      InputStream unbufIn = NetUtils.getInputStream(sock);
+      IOStreamPair ret = saslClient.newSocketSend(sock, unbufOut, unbufIn, this,
+        lb.getBlockToken(), dn);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeSocket(sock);
+      }
+    }
+  }
+  
+  /**
+   * Infer the checksum type for a replica by sending an OP_READ_BLOCK
+   * for the first byte of that replica. This is used for compatibility
+   * with older HDFS versions which did not include the checksum type in
+   * OpBlockChecksumResponseProto.
+   *
+   * @param lb the located block
+   * @param dn the connected datanode
+   * @return the inferred checksum type
+   * @throws IOException if an error occurs
+   */
+  private Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
+      throws IOException {
+    IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(), lb);
+
+    try {
+      DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
+          smallBufferSize));
+      DataInputStream in = new DataInputStream(pair.in);
+  
+      new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
+          0, 1, true, CachingStrategy.newDefaultStrategy());
+      final BlockOpResponseProto reply =
+          BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+      String logInfo = "trying to read " + lb.getBlock() + " from datanode " + dn;
+      DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
+      return PBHelperClient.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
+    } finally {
+      IOUtilsClient.cleanup(null, pair.in, pair.out);
+    }
+  }
+
+  /**
+   * Set permissions to a file or directory.
+   * @param src path name.
+   * @param permission permission to set to
+   * 
+   * @see ClientProtocol#setPermission(String, FsPermission)
+   */
+  public void setPermission(String src, FsPermission permission)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("setPermission", src);
+    try {
+      namenode.setPermission(src, permission);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Set file or directory owner.
+   * @param src path name.
+   * @param username user id.
+   * @param groupname user group.
+   * 
+   * @see ClientProtocol#setOwner(String, String, String)
+   */
+  public void setOwner(String src, String username, String groupname)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("setOwner", src);
+    try {
+      namenode.setOwner(src, username, groupname);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);                                   
+    } finally {
+      scope.close();
+    }
+  }
+
+  private long[] callGetStats() throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("getStats", traceSampler);
+    try {
+      return namenode.getStats();
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * @see ClientProtocol#getStats()
+   */
+  public FsStatus getDiskStatus() throws IOException {
+    long rawNums[] = callGetStats();
+    return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
+  }
+
+  /**
+   * Returns count of blocks with no good replicas left. Normally should be 
+   * zero.
+   * @throws IOException
+   */ 
+  public long getMissingBlocksCount() throws IOException {
+    return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
+  }
+  
+  /**
+   * Returns count of blocks with replication factor 1 and have
+   * lost the only replica.
+   * @throws IOException
+   */
+  public long getMissingReplOneBlocksCount() throws IOException {
+    return callGetStats()[ClientProtocol.
+        GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
+  }
+
+  /**
+   * Returns count of blocks with one of more replica missing.
+   * @throws IOException
+   */ 
+  public long getUnderReplicatedBlocksCount() throws IOException {
+    return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
+  }
+  
+  /**
+   * Returns count of blocks with at least one replica marked corrupt. 
+   * @throws IOException
+   */ 
+  public long getCorruptBlocksCount() throws IOException {
+    return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
+  }
+  
+  /**
+   * @return a list in which each entry describes a corrupt file/block
+   * @throws IOException
+   */
+  public CorruptFileBlocks listCorruptFileBlocks(String path,
+                                                 String cookie)
+        throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("listCorruptFileBlocks", path);
+    try {
+      return namenode.listCorruptFileBlocks(path, cookie);
+    } finally {
+      scope.close();
+    }
+  }
+
+  public DatanodeInfo[] datanodeReport(DatanodeReportType type)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("datanodeReport", traceSampler);
+    try {
+      return namenode.getDatanodeReport(type);
+    } finally {
+      scope.close();
+    }
+  }
+    
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      DatanodeReportType type) throws IOException {
+    checkOpen();
+    TraceScope scope =
+        Trace.startSpan("datanodeStorageReport", traceSampler);
+    try {
+      return namenode.getDatanodeStorageReport(type);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Enter, leave or get safe mode.
+   * 
+   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,boolean)
+   */
+  public boolean setSafeMode(SafeModeAction action) throws IOException {
+    return setSafeMode(action, false);
+  }
+  
+  /**
+   * Enter, leave or get safe mode.
+   * 
+   * @param action
+   *          One of SafeModeAction.GET, SafeModeAction.ENTER and
+   *          SafeModeActiob.LEAVE
+   * @param isChecked
+   *          If true, then check only active namenode's safemode status, else
+   *          check first namenode's status.
+   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
+   */
+  public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
+    TraceScope scope =
+        Trace.startSpan("setSafeMode", traceSampler);
+    try {
+      return namenode.setSafeMode(action, isChecked);
+    } finally {
+      scope.close();
+    }
+  }
+ 
+  /**
+   * Create one snapshot.
+   * 
+   * @param snapshotRoot The directory where the snapshot is to be taken
+   * @param snapshotName Name of the snapshot
+   * @return the snapshot path.
+   * @see ClientProtocol#createSnapshot(String, String)
+   */
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("createSnapshot", traceSampler);
+    try {
+      return namenode.createSnapshot(snapshotRoot, snapshotName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Delete a snapshot of a snapshottable directory.
+   * 
+   * @param snapshotRoot The snapshottable directory that the 
+   *                    to-be-deleted snapshot belongs to
+   * @param snapshotName The name of the to-be-deleted snapshot
+   * @throws IOException
+   * @see ClientProtocol#deleteSnapshot(String, String)
+   */
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("deleteSnapshot", traceSampler);
+    try {
+      namenode.deleteSnapshot(snapshotRoot, snapshotName);
+    } catch(R

<TRUNCATED>

[09/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
new file mode 100755
index 0000000..5466936
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -0,0 +1,923 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.channels.ClosedChannelException;
+import java.util.EnumSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.CanSetDropBehind;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSOutputSummer;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DataChecksum.Type;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+
+/****************************************************************
+ * DFSOutputStream creates files from a stream of bytes.
+ *
+ * The client application writes data that is cached internally by
+ * this stream. Data is broken up into packets, each packet is
+ * typically 64K in size. A packet comprises of chunks. Each chunk
+ * is typically 512 bytes and has an associated checksum with it.
+ *
+ * When a client application fills up the currentPacket, it is
+ * enqueued into the dataQueue of DataStreamer. DataStreamer is a
+ * thread that picks up packets from the dataQueue and sends it to
+ * the first datanode in the pipeline.
+ *
+ ****************************************************************/
+@InterfaceAudience.Private
+public class DFSOutputStream extends FSOutputSummer
+    implements Syncable, CanSetDropBehind {
+  static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
+  /**
+   * Number of times to retry creating a file when there are transient 
+   * errors (typically related to encryption zones and KeyProvider operations).
+   */
+  @VisibleForTesting
+  static final int CREATE_RETRY_COUNT = 10;
+  @VisibleForTesting
+  static CryptoProtocolVersion[] SUPPORTED_CRYPTO_VERSIONS =
+      CryptoProtocolVersion.supported();
+
+  protected final DFSClient dfsClient;
+  protected final ByteArrayManager byteArrayManager;
+  // closed is accessed by different threads under different locks.
+  protected volatile boolean closed = false;
+
+  protected final String src;
+  protected final long fileId;
+  protected final long blockSize;
+  protected final int bytesPerChecksum;
+
+  protected DFSPacket currentPacket = null;
+  private DataStreamer streamer;
+  protected int packetSize = 0; // write packet size, not including the header.
+  protected int chunksPerPacket = 0;
+  protected long lastFlushOffset = 0; // offset when flush was invoked
+  private long initialFileSize = 0; // at time of file open
+  private final short blockReplication; // replication factor of file
+  protected boolean shouldSyncBlock = false; // force blocks to disk upon close
+  protected final AtomicReference<CachingStrategy> cachingStrategy;
+  private FileEncryptionInfo fileEncryptionInfo;
+
+  /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
+  protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
+      long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
+    final byte[] buf;
+    final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
+
+    try {
+      buf = byteArrayManager.newByteArray(bufferSize);
+    } catch (InterruptedException ie) {
+      final InterruptedIOException iioe = new InterruptedIOException(
+          "seqno=" + seqno);
+      iioe.initCause(ie);
+      throw iioe;
+    }
+
+    return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno,
+                         getChecksumSize(), lastPacketInBlock);
+  }
+
+  @Override
+  protected void checkClosed() throws IOException {
+    if (isClosed()) {
+      getStreamer().getLastException().throwException4Close();
+    }
+  }
+
+  //
+  // returns the list of targets, if any, that is being currently used.
+  //
+  @VisibleForTesting
+  public synchronized DatanodeInfo[] getPipeline() {
+    if (getStreamer().streamerClosed()) {
+      return null;
+    }
+    DatanodeInfo[] currentNodes = getStreamer().getNodes();
+    if (currentNodes == null) {
+      return null;
+    }
+    DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
+    for (int i = 0; i < currentNodes.length; i++) {
+      value[i] = currentNodes[i];
+    }
+    return value;
+  }
+
+  /** 
+   * @return the object for computing checksum.
+   *         The type is NULL if checksum is not computed.
+   */
+  private static DataChecksum getChecksum4Compute(DataChecksum checksum,
+      HdfsFileStatus stat) {
+    if (DataStreamer.isLazyPersist(stat) && stat.getReplication() == 1) {
+      // do not compute checksum for writing to single replica to memory
+      return DataChecksum.newDataChecksum(Type.NULL,
+          checksum.getBytesPerChecksum());
+    }
+    return checksum;
+  }
+ 
+  private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
+      HdfsFileStatus stat, DataChecksum checksum) throws IOException {
+    super(getChecksum4Compute(checksum, stat));
+    this.dfsClient = dfsClient;
+    this.src = src;
+    this.fileId = stat.getFileId();
+    this.blockSize = stat.getBlockSize();
+    this.blockReplication = stat.getReplication();
+    this.fileEncryptionInfo = stat.getFileEncryptionInfo();
+    this.cachingStrategy = new AtomicReference<CachingStrategy>(
+        dfsClient.getDefaultWriteCachingStrategy());
+    if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug(
+          "Set non-null progress callback on DFSOutputStream " + src);
+    }
+    
+    this.bytesPerChecksum = checksum.getBytesPerChecksum();
+    if (bytesPerChecksum <= 0) {
+      throw new HadoopIllegalArgumentException(
+          "Invalid value: bytesPerChecksum = " + bytesPerChecksum + " <= 0");
+    }
+    if (blockSize % bytesPerChecksum != 0) {
+      throw new HadoopIllegalArgumentException("Invalid values: "
+          + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
+          + ") must divide block size (=" + blockSize + ").");
+    }
+    this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
+  }
+
+  /** Construct a new output stream for creating a file. */
+  protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
+      EnumSet<CreateFlag> flag, Progressable progress,
+      DataChecksum checksum, String[] favoredNodes) throws IOException {
+    this(dfsClient, src, progress, stat, checksum);
+    this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
+
+    computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
+
+    streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
+        cachingStrategy, byteArrayManager, favoredNodes);
+  }
+
+  static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
+      FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
+      short replication, long blockSize, Progressable progress, int buffersize,
+      DataChecksum checksum, String[] favoredNodes) throws IOException {
+    TraceScope scope =
+        dfsClient.getPathTraceScope("newStreamForCreate", src);
+    try {
+      HdfsFileStatus stat = null;
+
+      // Retry the create if we get a RetryStartFileException up to a maximum
+      // number of times
+      boolean shouldRetry = true;
+      int retryCount = CREATE_RETRY_COUNT;
+      while (shouldRetry) {
+        shouldRetry = false;
+        try {
+          stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
+              new EnumSetWritable<CreateFlag>(flag), createParent, replication,
+              blockSize, SUPPORTED_CRYPTO_VERSIONS);
+          break;
+        } catch (RemoteException re) {
+          IOException e = re.unwrapRemoteException(
+              AccessControlException.class,
+              DSQuotaExceededException.class,
+              QuotaByStorageTypeExceededException.class,
+              FileAlreadyExistsException.class,
+              FileNotFoundException.class,
+              ParentNotDirectoryException.class,
+              NSQuotaExceededException.class,
+              RetryStartFileException.class,
+              SafeModeException.class,
+              UnresolvedPathException.class,
+              SnapshotAccessControlException.class,
+              UnknownCryptoProtocolVersionException.class);
+          if (e instanceof RetryStartFileException) {
+            if (retryCount > 0) {
+              shouldRetry = true;
+              retryCount--;
+            } else {
+              throw new IOException("Too many retries because of encryption" +
+                  " zone operations", e);
+            }
+          } else {
+            throw e;
+          }
+        }
+      }
+      Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
+      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
+          flag, progress, checksum, favoredNodes);
+      out.start();
+      return out;
+    } finally {
+      scope.close();
+    }
+  }
+
+  /** Construct a new output stream for append. */
+  private DFSOutputStream(DFSClient dfsClient, String src,
+      EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
+      HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
+          throws IOException {
+    this(dfsClient, src, progress, stat, checksum);
+    initialFileSize = stat.getLen(); // length of file when opened
+    this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
+
+    boolean toNewBlock = flags.contains(CreateFlag.NEW_BLOCK);
+
+    this.fileEncryptionInfo = stat.getFileEncryptionInfo();
+
+    // The last partial block of the file has to be filled.
+    if (!toNewBlock && lastBlock != null) {
+      // indicate that we are appending to an existing block
+      streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
+          cachingStrategy, byteArrayManager);
+      getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
+      adjustPacketChunkSize(stat);
+      getStreamer().setPipelineInConstruction(lastBlock);
+    } else {
+      computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
+          bytesPerChecksum);
+      streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
+          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+          favoredNodes);
+    }
+  }
+
+  private void adjustPacketChunkSize(HdfsFileStatus stat) throws IOException{
+
+    long usedInLastBlock = stat.getLen() % blockSize;
+    int freeInLastBlock = (int)(blockSize - usedInLastBlock);
+
+    // calculate the amount of free space in the pre-existing
+    // last crc chunk
+    int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
+    int freeInCksum = bytesPerChecksum - usedInCksum;
+
+    // if there is space in the last block, then we have to
+    // append to that block
+    if (freeInLastBlock == blockSize) {
+      throw new IOException("The last block for file " +
+          src + " is full.");
+    }
+
+    if (usedInCksum > 0 && freeInCksum > 0) {
+      // if there is space in the last partial chunk, then
+      // setup in such a way that the next packet will have only
+      // one chunk that fills up the partial chunk.
+      //
+      computePacketChunkSize(0, freeInCksum);
+      setChecksumBufSize(freeInCksum);
+      getStreamer().setAppendChunk(true);
+    } else {
+      // if the remaining space in the block is smaller than
+      // that expected size of of a packet, then create
+      // smaller size packet.
+      //
+      computePacketChunkSize(
+          Math.min(dfsClient.getConf().getWritePacketSize(), freeInLastBlock),
+          bytesPerChecksum);
+    }
+  }
+
+  static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
+      EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
+      LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
+      String[] favoredNodes) throws IOException {
+    TraceScope scope =
+        dfsClient.getPathTraceScope("newStreamForAppend", src);
+    try {
+      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
+          progress, lastBlock, stat, checksum, favoredNodes);
+      out.start();
+      return out;
+    } finally {
+      scope.close();
+    }
+  }
+
+  protected void computePacketChunkSize(int psize, int csize) {
+    final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
+    final int chunkSize = csize + getChecksumSize();
+    chunksPerPacket = Math.max(bodySize/chunkSize, 1);
+    packetSize = chunkSize*chunksPerPacket;
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("computePacketChunkSize: src=" + src +
+                ", chunkSize=" + chunkSize +
+                ", chunksPerPacket=" + chunksPerPacket +
+                ", packetSize=" + packetSize);
+    }
+  }
+
+  protected TraceScope createWriteTraceScope() {
+    return dfsClient.getPathTraceScope("DFSOutputStream#write", src);
+  }
+
+  // @see FSOutputSummer#writeChunk()
+  @Override
+  protected synchronized void writeChunk(byte[] b, int offset, int len,
+      byte[] checksum, int ckoff, int cklen) throws IOException {
+    dfsClient.checkOpen();
+    checkClosed();
+
+    if (len > bytesPerChecksum) {
+      throw new IOException("writeChunk() buffer size is " + len +
+                            " is larger than supported  bytesPerChecksum " +
+                            bytesPerChecksum);
+    }
+    if (cklen != 0 && cklen != getChecksumSize()) {
+      throw new IOException("writeChunk() checksum size is supposed to be " +
+                            getChecksumSize() + " but found to be " + cklen);
+    }
+
+    if (currentPacket == null) {
+      currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
+          .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + 
+            currentPacket.getSeqno() +
+            ", src=" + src +
+            ", packetSize=" + packetSize +
+            ", chunksPerPacket=" + chunksPerPacket +
+            ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
+      }
+    }
+
+    currentPacket.writeChecksum(checksum, ckoff, cklen);
+    currentPacket.writeData(b, offset, len);
+    currentPacket.incNumChunks();
+    getStreamer().incBytesCurBlock(len);
+
+    // If packet is full, enqueue it for transmission
+    //
+    if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
+        getStreamer().getBytesCurBlock() == blockSize) {
+      enqueueCurrentPacketFull();
+    }
+  }
+
+  void enqueueCurrentPacket() throws IOException {
+    getStreamer().waitAndQueuePacket(currentPacket);
+    currentPacket = null;
+  }
+
+  void enqueueCurrentPacketFull() throws IOException {
+    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
+        + " appendChunk={}, {}", currentPacket, src, getStreamer()
+        .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+        getStreamer());
+    enqueueCurrentPacket();
+    adjustChunkBoundary();
+    endBlock();
+  }
+
+  /** create an empty packet to mark the end of the block. */
+  void setCurrentPacketToEmpty() throws InterruptedIOException {
+    currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+        getStreamer().getAndIncCurrentSeqno(), true);
+    currentPacket.setSyncBlock(shouldSyncBlock);
+  }
+
+  /**
+   * If the reopened file did not end at chunk boundary and the above
+   * write filled up its partial chunk. Tell the summer to generate full
+   * crc chunks from now on.
+   */
+  protected void adjustChunkBoundary() {
+    if (getStreamer().getAppendChunk() &&
+        getStreamer().getBytesCurBlock() % bytesPerChecksum == 0) {
+      getStreamer().setAppendChunk(false);
+      resetChecksumBufSize();
+    }
+
+    if (!getStreamer().getAppendChunk()) {
+      int psize = Math.min((int)(blockSize- getStreamer().getBytesCurBlock()),
+          dfsClient.getConf().getWritePacketSize());
+      computePacketChunkSize(psize, bytesPerChecksum);
+    }
+  }
+
+  /**
+   * if encountering a block boundary, send an empty packet to
+   * indicate the end of block and reset bytesCurBlock.
+   *
+   * @throws IOException
+   */
+  protected void endBlock() throws IOException {
+    if (getStreamer().getBytesCurBlock() == blockSize) {
+      setCurrentPacketToEmpty();
+      enqueueCurrentPacket();
+      getStreamer().setBytesCurBlock(0);
+      lastFlushOffset = 0;
+    }
+  }
+
+  @Deprecated
+  public void sync() throws IOException {
+    hflush();
+  }
+  
+  /**
+   * Flushes out to all replicas of the block. The data is in the buffers
+   * of the DNs but not necessarily in the DN's OS buffers.
+   *
+   * It is a synchronous operation. When it returns,
+   * it guarantees that flushed data become visible to new readers. 
+   * It is not guaranteed that data has been flushed to 
+   * persistent store on the datanode. 
+   * Block allocations are persisted on namenode.
+   */
+  @Override
+  public void hflush() throws IOException {
+    TraceScope scope =
+        dfsClient.getPathTraceScope("hflush", src);
+    try {
+      flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
+    } finally {
+      scope.close();
+    }
+  }
+
+  @Override
+  public void hsync() throws IOException {
+    TraceScope scope =
+        dfsClient.getPathTraceScope("hsync", src);
+    try {
+      flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * The expected semantics is all data have flushed out to all replicas 
+   * and all replicas have done posix fsync equivalent - ie the OS has 
+   * flushed it to the disk device (but the disk may have it in its cache).
+   * 
+   * Note that only the current block is flushed to the disk device.
+   * To guarantee durable sync across block boundaries the stream should
+   * be created with {@link CreateFlag#SYNC_BLOCK}.
+   * 
+   * @param syncFlags
+   *          Indicate the semantic of the sync. Currently used to specify
+   *          whether or not to update the block length in NameNode.
+   */
+  public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
+    TraceScope scope =
+        dfsClient.getPathTraceScope("hsync", src);
+    try {
+      flushOrSync(true, syncFlags);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Flush/Sync buffered data to DataNodes.
+   * 
+   * @param isSync
+   *          Whether or not to require all replicas to flush data to the disk
+   *          device
+   * @param syncFlags
+   *          Indicate extra detailed semantic of the flush/sync. Currently
+   *          mainly used to specify whether or not to update the file length in
+   *          the NameNode
+   * @throws IOException
+   */
+  private void flushOrSync(boolean isSync, EnumSet<SyncFlag> syncFlags)
+      throws IOException {
+    dfsClient.checkOpen();
+    checkClosed();
+    try {
+      long toWaitFor;
+      long lastBlockLength = -1L;
+      boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
+      boolean endBlock = syncFlags.contains(SyncFlag.END_BLOCK);
+      synchronized (this) {
+        // flush checksum buffer, but keep checksum buffer intact if we do not
+        // need to end the current block
+        int numKept = flushBuffer(!endBlock, true);
+        // bytesCurBlock potentially incremented if there was buffered data
+
+        if (DFSClient.LOG.isDebugEnabled()) {
+          DFSClient.LOG.debug("DFSClient flush(): "
+              + " bytesCurBlock=" + getStreamer().getBytesCurBlock()
+              + " lastFlushOffset=" + lastFlushOffset
+              + " createNewBlock=" + endBlock);
+        }
+        // Flush only if we haven't already flushed till this offset.
+        if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
+          assert getStreamer().getBytesCurBlock() > lastFlushOffset;
+          // record the valid offset of this flush
+          lastFlushOffset = getStreamer().getBytesCurBlock();
+          if (isSync && currentPacket == null && !endBlock) {
+            // Nothing to send right now,
+            // but sync was requested.
+            // Send an empty packet if we do not end the block right now
+            currentPacket = createPacket(packetSize, chunksPerPacket,
+                getStreamer().getBytesCurBlock(), getStreamer()
+                    .getAndIncCurrentSeqno(), false);
+          }
+        } else {
+          if (isSync && getStreamer().getBytesCurBlock() > 0 && !endBlock) {
+            // Nothing to send right now,
+            // and the block was partially written,
+            // and sync was requested.
+            // So send an empty sync packet if we do not end the block right
+            // now
+            currentPacket = createPacket(packetSize, chunksPerPacket,
+                getStreamer().getBytesCurBlock(), getStreamer()
+                    .getAndIncCurrentSeqno(), false);
+          } else if (currentPacket != null) {
+            // just discard the current packet since it is already been sent.
+            currentPacket.releaseBuffer(byteArrayManager);
+            currentPacket = null;
+          }
+        }
+        if (currentPacket != null) {
+          currentPacket.setSyncBlock(isSync);
+          enqueueCurrentPacket();
+        }
+        if (endBlock && getStreamer().getBytesCurBlock() > 0) {
+          // Need to end the current block, thus send an empty packet to
+          // indicate this is the end of the block and reset bytesCurBlock
+          currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+              getStreamer().getAndIncCurrentSeqno(), true);
+          currentPacket.setSyncBlock(shouldSyncBlock || isSync);
+          enqueueCurrentPacket();
+          getStreamer().setBytesCurBlock(0);
+          lastFlushOffset = 0;
+        } else {
+          // Restore state of stream. Record the last flush offset
+          // of the last full chunk that was flushed.
+          getStreamer().setBytesCurBlock(
+              getStreamer().getBytesCurBlock() - numKept);
+        }
+
+        toWaitFor = getStreamer().getLastQueuedSeqno();
+      } // end synchronized
+
+      getStreamer().waitForAckedSeqno(toWaitFor);
+
+      // update the block length first time irrespective of flag
+      if (updateLength || getStreamer().getPersistBlocks().get()) {
+        synchronized (this) {
+          if (!getStreamer().streamerClosed()
+              && getStreamer().getBlock() != null) {
+            lastBlockLength = getStreamer().getBlock().getNumBytes();
+          }
+        }
+      }
+      // If 1) any new blocks were allocated since the last flush, or 2) to
+      // update length in NN is required, then persist block locations on
+      // namenode.
+      if (getStreamer().getPersistBlocks().getAndSet(false) || updateLength) {
+        try {
+          dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
+              lastBlockLength);
+        } catch (IOException ioe) {
+          DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
+          // If we got an error here, it might be because some other thread called
+          // close before our hflush completed. In that case, we should throw an
+          // exception that the stream is closed.
+          checkClosed();
+          // If we aren't closed but failed to sync, we should expose that to the
+          // caller.
+          throw ioe;
+        }
+      }
+
+      synchronized(this) {
+        if (!getStreamer().streamerClosed()) {
+          getStreamer().setHflush();
+        }
+      }
+    } catch (InterruptedIOException interrupt) {
+      // This kind of error doesn't mean that the stream itself is broken - just the
+      // flushing thread got interrupted. So, we shouldn't close down the writer,
+      // but instead just propagate the error
+      throw interrupt;
+    } catch (IOException e) {
+      DFSClient.LOG.warn("Error while syncing", e);
+      synchronized (this) {
+        if (!isClosed()) {
+          getStreamer().getLastException().set(e);
+          closeThreads(true);
+        }
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * @deprecated use {@link HdfsDataOutputStream#getCurrentBlockReplication()}.
+   */
+  @Deprecated
+  public synchronized int getNumCurrentReplicas() throws IOException {
+    return getCurrentBlockReplication();
+  }
+
+  /**
+   * Note that this is not a public API;
+   * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
+   * 
+   * @return the number of valid replicas of the current block
+   */
+  public synchronized int getCurrentBlockReplication() throws IOException {
+    dfsClient.checkOpen();
+    checkClosed();
+    if (getStreamer().streamerClosed()) {
+      return blockReplication; // no pipeline, return repl factor of file
+    }
+    DatanodeInfo[] currentNodes = getStreamer().getNodes();
+    if (currentNodes == null) {
+      return blockReplication; // no pipeline, return repl factor of file
+    }
+    return currentNodes.length;
+  }
+  
+  /**
+   * Waits till all existing data is flushed and confirmations 
+   * received from datanodes. 
+   */
+  protected void flushInternal() throws IOException {
+    long toWaitFor;
+    synchronized (this) {
+      dfsClient.checkOpen();
+      checkClosed();
+      //
+      // If there is data in the current buffer, send it across
+      //
+      getStreamer().queuePacket(currentPacket);
+      currentPacket = null;
+      toWaitFor = getStreamer().getLastQueuedSeqno();
+    }
+
+    getStreamer().waitForAckedSeqno(toWaitFor);
+  }
+
+  protected synchronized void start() {
+    getStreamer().start();
+  }
+  
+  /**
+   * Aborts this output stream and releases any system 
+   * resources associated with this stream.
+   */
+  synchronized void abort() throws IOException {
+    if (isClosed()) {
+      return;
+    }
+    getStreamer().getLastException().set(new IOException("Lease timeout of "
+        + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
+    closeThreads(true);
+    dfsClient.endFileLease(fileId);
+  }
+
+  boolean isClosed() {
+    return closed || getStreamer().streamerClosed();
+  }
+
+  void setClosed() {
+    closed = true;
+    getStreamer().release();
+  }
+
+  // shutdown datastreamer and responseprocessor threads.
+  // interrupt datastreamer if force is true
+  protected void closeThreads(boolean force) throws IOException {
+    try {
+      getStreamer().close(force);
+      getStreamer().join();
+      getStreamer().closeSocket();
+    } catch (InterruptedException e) {
+      throw new IOException("Failed to shutdown streamer");
+    } finally {
+      getStreamer().setSocketToNull();
+      setClosed();
+    }
+  }
+  
+  /**
+   * Closes this output stream and releases any system 
+   * resources associated with this stream.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    TraceScope scope =
+        dfsClient.getPathTraceScope("DFSOutputStream#close", src);
+    try {
+      closeImpl();
+    } finally {
+      scope.close();
+    }
+  }
+
+  protected synchronized void closeImpl() throws IOException {
+    if (isClosed()) {
+      getStreamer().getLastException().check(true);
+      return;
+    }
+
+    try {
+      flushBuffer();       // flush from all upper layers
+
+      if (currentPacket != null) {
+        enqueueCurrentPacket();
+      }
+
+      if (getStreamer().getBytesCurBlock() != 0) {
+        setCurrentPacketToEmpty();
+      }
+
+      flushInternal();             // flush all data to Datanodes
+      // get last block before destroying the streamer
+      ExtendedBlock lastBlock = getStreamer().getBlock();
+      closeThreads(false);
+      TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
+      try {
+        completeFile(lastBlock);
+      } finally {
+        scope.close();
+      }
+      dfsClient.endFileLease(fileId);
+    } catch (ClosedChannelException e) {
+    } finally {
+      setClosed();
+    }
+  }
+
+  // should be called holding (this) lock since setTestFilename() may 
+  // be called during unit tests
+  protected void completeFile(ExtendedBlock last) throws IOException {
+    long localstart = Time.monotonicNow();
+    final DfsClientConf conf = dfsClient.getConf();
+    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+    boolean fileComplete = false;
+    int retries = conf.getNumBlockWriteLocateFollowingRetry();
+    while (!fileComplete) {
+      fileComplete =
+          dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
+      if (!fileComplete) {
+        final int hdfsTimeout = conf.getHdfsTimeout();
+        if (!dfsClient.clientRunning
+            || (hdfsTimeout > 0
+                && localstart + hdfsTimeout < Time.monotonicNow())) {
+            String msg = "Unable to close file because dfsclient " +
+                          " was unable to contact the HDFS servers." +
+                          " clientRunning " + dfsClient.clientRunning +
+                          " hdfsTimeout " + hdfsTimeout;
+            DFSClient.LOG.info(msg);
+            throw new IOException(msg);
+        }
+        try {
+          if (retries == 0) {
+            throw new IOException("Unable to close file because the last block"
+                + " does not have enough number of replicas.");
+          }
+          retries--;
+          Thread.sleep(sleeptime);
+          sleeptime *= 2;
+          if (Time.monotonicNow() - localstart > 5000) {
+            DFSClient.LOG.info("Could not complete " + src + " retrying...");
+          }
+        } catch (InterruptedException ie) {
+          DFSClient.LOG.warn("Caught exception ", ie);
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public void setArtificialSlowdown(long period) {
+    getStreamer().setArtificialSlowdown(period);
+  }
+
+  @VisibleForTesting
+  public synchronized void setChunksPerPacket(int value) {
+    chunksPerPacket = Math.min(chunksPerPacket, value);
+    packetSize = (bytesPerChecksum + getChecksumSize()) * chunksPerPacket;
+  }
+
+  /**
+   * Returns the size of a file as it was when this stream was opened
+   */
+  public long getInitialLen() {
+    return initialFileSize;
+  }
+
+  /**
+   * @return the FileEncryptionInfo for this stream, or null if not encrypted.
+   */
+  public FileEncryptionInfo getFileEncryptionInfo() {
+    return fileEncryptionInfo;
+  }
+
+  /**
+   * Returns the access token currently used by streamer, for testing only
+   */
+  synchronized Token<BlockTokenIdentifier> getBlockToken() {
+    return getStreamer().getBlockToken();
+  }
+
+  @Override
+  public void setDropBehind(Boolean dropBehind) throws IOException {
+    CachingStrategy prevStrategy, nextStrategy;
+    // CachingStrategy is immutable.  So build a new CachingStrategy with the
+    // modifications we want, and compare-and-swap it in.
+    do {
+      prevStrategy = this.cachingStrategy.get();
+      nextStrategy = new CachingStrategy.Builder(prevStrategy).
+                        setDropBehind(dropBehind).build();
+    } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
+  }
+
+  @VisibleForTesting
+  ExtendedBlock getBlock() {
+    return getStreamer().getBlock();
+  }
+
+  @VisibleForTesting
+  public long getFileId() {
+    return fileId;
+  }
+
+  /**
+   * Return the source of stream.
+   */
+  String getSrc() {
+    return src;
+  }
+
+  /**
+   * Returns the data streamer object.
+   */
+  protected DataStreamer getStreamer() {
+    return streamer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
new file mode 100755
index 0000000..22055c3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
@@ -0,0 +1,345 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.channels.ClosedChannelException;
+import java.util.Arrays;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.htrace.Span;
+
+/****************************************************************
+ * DFSPacket is used by DataStreamer and DFSOutputStream.
+ * DFSOutputStream generates packets and then ask DatStreamer
+ * to send them to datanodes.
+ ****************************************************************/
+
+@InterfaceAudience.Private
+class DFSPacket {
+  public static final long HEART_BEAT_SEQNO = -1L;
+  private static long[] EMPTY = new long[0];
+  private final long seqno; // sequence number of buffer in block
+  private final long offsetInBlock; // offset in block
+  private boolean syncBlock; // this packet forces the current block to disk
+  private int numChunks; // number of chunks currently in packet
+  private final int maxChunks; // max chunks in packet
+  private byte[] buf;
+  private final boolean lastPacketInBlock; // is this the last packet in block?
+
+  /**
+   * buf is pointed into like follows:
+   *  (C is checksum data, D is payload data)
+   *
+   * [_________CCCCCCCCC________________DDDDDDDDDDDDDDDD___]
+   *           ^        ^               ^               ^
+   *           |        checksumPos     dataStart       dataPos
+   *           checksumStart
+   *
+   * Right before sending, we move the checksum data to immediately precede
+   * the actual data, and then insert the header into the buffer immediately
+   * preceding the checksum data, so we make sure to keep enough space in
+   * front of the checksum data to support the largest conceivable header.
+   */
+  private int checksumStart;
+  private int checksumPos;
+  private final int dataStart;
+  private int dataPos;
+  private long[] traceParents = EMPTY;
+  private int traceParentsUsed;
+  private Span span;
+
+  /**
+   * Create a new packet.
+   *
+   * @param buf the buffer storing data and checksums
+   * @param chunksPerPkt maximum number of chunks per packet.
+   * @param offsetInBlock offset in bytes into the HDFS block.
+   * @param seqno the sequence number of this packet
+   * @param checksumSize the size of checksum
+   * @param lastPacketInBlock if this is the last packet
+   */
+  DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
+                   int checksumSize, boolean lastPacketInBlock) {
+    this.lastPacketInBlock = lastPacketInBlock;
+    this.numChunks = 0;
+    this.offsetInBlock = offsetInBlock;
+    this.seqno = seqno;
+
+    this.buf = buf;
+
+    checksumStart = PacketHeader.PKT_MAX_HEADER_LEN;
+    checksumPos = checksumStart;
+    dataStart = checksumStart + (chunksPerPkt * checksumSize);
+    dataPos = dataStart;
+    maxChunks = chunksPerPkt;
+  }
+
+  /**
+   * Write data to this packet.
+   *
+   * @param inarray input array of data
+   * @param off the offset of data to write
+   * @param len the length of data to write
+   * @throws ClosedChannelException
+   */
+  synchronized void writeData(byte[] inarray, int off, int len)
+      throws ClosedChannelException {
+    checkBuffer();
+    if (dataPos + len > buf.length) {
+      throw new BufferOverflowException();
+    }
+    System.arraycopy(inarray, off, buf, dataPos, len);
+    dataPos += len;
+  }
+
+  /**
+   * Write checksums to this packet
+   *
+   * @param inarray input array of checksums
+   * @param off the offset of checksums to write
+   * @param len the length of checksums to write
+   * @throws ClosedChannelException
+   */
+  synchronized void writeChecksum(byte[] inarray, int off, int len)
+      throws ClosedChannelException {
+    checkBuffer();
+    if (len == 0) {
+      return;
+    }
+    if (checksumPos + len > dataStart) {
+      throw new BufferOverflowException();
+    }
+    System.arraycopy(inarray, off, buf, checksumPos, len);
+    checksumPos += len;
+  }
+
+  /**
+   * Write the full packet, including the header, to the given output stream.
+   *
+   * @param stm
+   * @throws IOException
+   */
+  synchronized void writeTo(DataOutputStream stm) throws IOException {
+    checkBuffer();
+
+    final int dataLen = dataPos - dataStart;
+    final int checksumLen = checksumPos - checksumStart;
+    final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen;
+
+    PacketHeader header = new PacketHeader(
+        pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
+
+    if (checksumPos != dataStart) {
+      // Move the checksum to cover the gap. This can happen for the last
+      // packet or during an hflush/hsync call.
+      System.arraycopy(buf, checksumStart, buf,
+          dataStart - checksumLen , checksumLen);
+      checksumPos = dataStart;
+      checksumStart = checksumPos - checksumLen;
+    }
+
+    final int headerStart = checksumStart - header.getSerializedSize();
+    assert checksumStart + 1 >= header.getSerializedSize();
+    assert headerStart >= 0;
+    assert headerStart + header.getSerializedSize() == checksumStart;
+
+    // Copy the header data into the buffer immediately preceding the checksum
+    // data.
+    System.arraycopy(header.getBytes(), 0, buf, headerStart,
+        header.getSerializedSize());
+
+    // corrupt the data for testing.
+    if (DFSClientFaultInjector.get().corruptPacket()) {
+      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+    }
+
+    // Write the now contiguous full packet to the output stream.
+    stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
+
+    // undo corruption.
+    if (DFSClientFaultInjector.get().uncorruptPacket()) {
+      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+    }
+  }
+
+  private synchronized void checkBuffer() throws ClosedChannelException {
+    if (buf == null) {
+      throw new ClosedChannelException();
+    }
+  }
+
+  /**
+   * Release the buffer in this packet to ByteArrayManager.
+   *
+   * @param bam
+   */
+  synchronized void releaseBuffer(ByteArrayManager bam) {
+    bam.release(buf);
+    buf = null;
+  }
+
+  /**
+   * get the packet's last byte's offset in the block
+   *
+   * @return the packet's last byte's offset in the block
+   */
+  synchronized long getLastByteOffsetBlock() {
+    return offsetInBlock + dataPos - dataStart;
+  }
+
+  /**
+   * Check if this packet is a heart beat packet
+   *
+   * @return true if the sequence number is HEART_BEAT_SEQNO
+   */
+  boolean isHeartbeatPacket() {
+    return seqno == HEART_BEAT_SEQNO;
+  }
+
+  /**
+   * check if this packet is the last packet in block
+   *
+   * @return true if the packet is the last packet
+   */
+  boolean isLastPacketInBlock(){
+    return lastPacketInBlock;
+  }
+
+  /**
+   * get sequence number of this packet
+   *
+   * @return the sequence number of this packet
+   */
+  long getSeqno(){
+    return seqno;
+  }
+
+  /**
+   * get the number of chunks this packet contains
+   *
+   * @return the number of chunks in this packet
+   */
+  synchronized int getNumChunks(){
+    return numChunks;
+  }
+
+  /**
+   * increase the number of chunks by one
+   */
+  synchronized void incNumChunks(){
+    numChunks++;
+  }
+
+  /**
+   * get the maximum number of packets
+   *
+   * @return the maximum number of packets
+   */
+  int getMaxChunks(){
+    return maxChunks;
+  }
+
+  /**
+   * set if to sync block
+   *
+   * @param syncBlock if to sync block
+   */
+  synchronized void setSyncBlock(boolean syncBlock){
+    this.syncBlock = syncBlock;
+  }
+
+  @Override
+  public String toString() {
+    return "packet seqno: " + this.seqno +
+        " offsetInBlock: " + this.offsetInBlock +
+        " lastPacketInBlock: " + this.lastPacketInBlock +
+        " lastByteOffsetInBlock: " + this.getLastByteOffsetBlock();
+  }
+
+  /**
+   * Add a trace parent span for this packet.<p/>
+   *
+   * Trace parent spans for a packet are the trace spans responsible for
+   * adding data to that packet.  We store them as an array of longs for
+   * efficiency.<p/>
+   *
+   * Protected by the DFSOutputStream dataQueue lock.
+   */
+  public void addTraceParent(Span span) {
+    if (span == null) {
+      return;
+    }
+    addTraceParent(span.getSpanId());
+  }
+
+  public void addTraceParent(long id) {
+    if (traceParentsUsed == traceParents.length) {
+      int newLength = (traceParents.length == 0) ? 8 :
+          traceParents.length * 2;
+      traceParents = Arrays.copyOf(traceParents, newLength);
+    }
+    traceParents[traceParentsUsed] = id;
+    traceParentsUsed++;
+  }
+
+  /**
+   * Get the trace parent spans for this packet.<p/>
+   *
+   * Will always be non-null.<p/>
+   *
+   * Protected by the DFSOutputStream dataQueue lock.
+   */
+  public long[] getTraceParents() {
+    // Remove duplicates from the array.
+    int len = traceParentsUsed;
+    Arrays.sort(traceParents, 0, len);
+    int i = 0, j = 0;
+    long prevVal = 0; // 0 is not a valid span id
+    while (true) {
+      if (i == len) {
+        break;
+      }
+      long val = traceParents[i];
+      if (val != prevVal) {
+        traceParents[j] = val;
+        j++;
+        prevVal = val;
+      }
+      i++;
+    }
+    if (j < traceParents.length) {
+      traceParents = Arrays.copyOf(traceParents, j);
+      traceParentsUsed = traceParents.length;
+    }
+    return traceParents;
+  }
+
+  public void setTraceSpan(Span span) {
+    this.span = span;
+  }
+
+  public Span getTraceSpan() {
+    return span;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 359886e..e275afb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -590,6 +591,29 @@ public class DFSUtilClient {
     }
   }
 
+  public static int getIoFileBufferSize(Configuration conf) {
+    return conf.getInt(
+        CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+        CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+  }
+
+  public static int getSmallBufferSize(Configuration conf) {
+    return Math.min(getIoFileBufferSize(conf) / 2, 512);
+  }
+
+  /**
+   * Probe for HDFS Encryption being enabled; this uses the value of
+   * the option {@link HdfsClientConfigKeys#DFS_ENCRYPTION_KEY_PROVIDER_URI},
+   * returning true if that property contains a non-empty, non-whitespace
+   * string.
+   * @param conf configuration to probe
+   * @return true if encryption is considered enabled.
+   */
+  public static boolean isHDFSEncryptionEnabled(Configuration conf) {
+    return !conf.getTrimmed(
+        HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty();
+  }
+
   public static InetSocketAddress getNNAddress(String address) {
     return NetUtils.createSocketAddr(address,
         HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);


[07/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
new file mode 100644
index 0000000..745ca7e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.crypto.CryptoOutputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The Hdfs implementation of {@link FSDataOutputStream}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HdfsDataOutputStream extends FSDataOutputStream {
+  public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats,
+      long startPosition) throws IOException {
+    super(out, stats, startPosition);
+  }
+
+  public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
+      ) throws IOException {
+    this(out, stats, 0L);
+  }
+
+  public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
+      long startPosition) throws IOException {
+    super(out, stats, startPosition);
+    Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
+        "CryptoOutputStream should wrap a DFSOutputStream");
+  }
+
+  public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
+      throws IOException {
+    this(out, stats, 0L);
+  }
+
+  /**
+   * Get the actual number of replicas of the current block.
+   * 
+   * This can be different from the designated replication factor of the file
+   * because the namenode does not maintain replication for the blocks which are
+   * currently being written to. Depending on the configuration, the client may
+   * continue to write to a block even if a few datanodes in the write pipeline
+   * have failed, or the client may add a new datanodes once a datanode has
+   * failed.
+   * 
+   * @return the number of valid replicas of the current block
+   */
+  public synchronized int getCurrentBlockReplication() throws IOException {
+    OutputStream wrappedStream = getWrappedStream();
+    if (wrappedStream instanceof CryptoOutputStream) {
+      wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
+    }
+    return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
+  }
+  
+  /**
+   * Sync buffered data to DataNodes (flush to disk devices).
+   * 
+   * @param syncFlags
+   *          Indicate the detailed semantic and actions of the hsync.
+   * @throws IOException
+   * @see FSDataOutputStream#hsync()
+   */
+  public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
+    OutputStream wrappedStream = getWrappedStream();
+    if (wrappedStream instanceof CryptoOutputStream) {
+      ((CryptoOutputStream) wrappedStream).flush();
+      wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
+    }
+    ((DFSOutputStream) wrappedStream).hsync(syncFlags);
+  }
+  
+  public static enum SyncFlag {
+
+    /**
+     * When doing sync to DataNodes, also update the metadata (block length) in
+     * the NameNode.
+     */
+    UPDATE_LENGTH,
+
+    /**
+     * Sync the data to DataNode, close the current block, and allocate a new
+     * block
+     */
+    END_BLOCK;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
new file mode 100644
index 0000000..c3d2cfc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
@@ -0,0 +1,524 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client.impl;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing file-being-written leases
+ * on the namenode.
+ * When a file is opened for write (create or append),
+ * namenode stores a file lease for recording the identity of the writer.
+ * The writer (i.e. the DFSClient) is required to renew the lease periodically.
+ * When the lease is not renewed before it expires,
+ * the namenode considers the writer as failed and then it may either let
+ * another writer to obtain the lease or close the file.
+ * </p>
+ * <p>
+ * This class also provides the following functionality:
+ * <ul>
+ * <li>
+ * It maintains a map from (namenode, user) pairs to lease renewers.
+ * The same {@link LeaseRenewer} instance is used for renewing lease
+ * for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and the same user.
+ * </li>
+ * <li>
+ * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
+ * Periodically the leases for all the clients are renewed.
+ * A client is removed from the list when the client is closed.
+ * </li>
+ * <li>
+ * A thread per namenode per user is used by the {@link LeaseRenewer}
+ * to renew the leases.
+ * </li>
+ * </ul>
+ * </p>
+ */
+@InterfaceAudience.Private
+public class LeaseRenewer {
+  static final Logger LOG = LoggerFactory.getLogger(LeaseRenewer.class);
+
+  static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L;
+  static final long LEASE_RENEWER_SLEEP_DEFAULT = 1000L;
+
+  /** Get a {@link LeaseRenewer} instance */
+  public static LeaseRenewer getInstance(final String authority,
+      final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
+    final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
+    r.addClient(dfsc);
+    return r;
+  }
+
+  /**
+   * A factory for sharing {@link LeaseRenewer} objects
+   * among {@link DFSClient} instances
+   * so that there is only one renewer per authority per user.
+   */
+  private static class Factory {
+    private static final Factory INSTANCE = new Factory();
+
+    private static class Key {
+      /** Namenode info */
+      final String authority;
+      /** User info */
+      final UserGroupInformation ugi;
+
+      private Key(final String authority, final UserGroupInformation ugi) {
+        if (authority == null) {
+          throw new HadoopIllegalArgumentException("authority == null");
+        } else if (ugi == null) {
+          throw new HadoopIllegalArgumentException("ugi == null");
+        }
+
+        this.authority = authority;
+        this.ugi = ugi;
+      }
+
+      @Override
+      public int hashCode() {
+        return authority.hashCode() ^ ugi.hashCode();
+      }
+
+      @Override
+      public boolean equals(Object obj) {
+        if (obj == this) {
+          return true;
+        }
+        if (obj != null && obj instanceof Key) {
+          final Key that = (Key)obj;
+          return this.authority.equals(that.authority)
+                 && this.ugi.equals(that.ugi);
+        }
+        return false;
+      }
+
+      @Override
+      public String toString() {
+        return ugi.getShortUserName() + "@" + authority;
+      }
+    }
+
+    /** A map for per user per namenode renewers. */
+    private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
+
+    /** Get a renewer. */
+    private synchronized LeaseRenewer get(final String authority,
+        final UserGroupInformation ugi) {
+      final Key k = new Key(authority, ugi);
+      LeaseRenewer r = renewers.get(k);
+      if (r == null) {
+        r = new LeaseRenewer(k);
+        renewers.put(k, r);
+      }
+      return r;
+    }
+
+    /** Remove the given renewer. */
+    private synchronized void remove(final LeaseRenewer r) {
+      final LeaseRenewer stored = renewers.get(r.factorykey);
+      //Since a renewer may expire, the stored renewer can be different.
+      if (r == stored) {
+        if (!r.clientsRunning()) {
+          renewers.remove(r.factorykey);
+        }
+      }
+    }
+  }
+
+  /** The time in milliseconds that the map became empty. */
+  private long emptyTime = Long.MAX_VALUE;
+  /** A fixed lease renewal time period in milliseconds */
+  private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD / 2;
+
+  /** A daemon for renewing lease */
+  private Daemon daemon = null;
+  /** Only the daemon with currentId should run. */
+  private int currentId = 0;
+
+  /**
+   * A period in milliseconds that the lease renewer thread should run
+   * after the map became empty.
+   * In other words,
+   * if the map is empty for a time period longer than the grace period,
+   * the renewer should terminate.
+   */
+  private long gracePeriod;
+  /**
+   * The time period in milliseconds
+   * that the renewer sleeps for each iteration.
+   */
+  private long sleepPeriod;
+
+  private final Factory.Key factorykey;
+
+  /** A list of clients corresponding to this renewer. */
+  private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
+
+  /**
+   * A stringified stack trace of the call stack when the Lease Renewer
+   * was instantiated. This is only generated if trace-level logging is
+   * enabled on this class.
+   */
+  private final String instantiationTrace;
+
+  private LeaseRenewer(Factory.Key factorykey) {
+    this.factorykey = factorykey;
+    unsyncSetGraceSleepPeriod(LEASE_RENEWER_GRACE_DEFAULT);
+
+    if (LOG.isTraceEnabled()) {
+      instantiationTrace = StringUtils.stringifyException(
+        new Throwable("TRACE"));
+    } else {
+      instantiationTrace = null;
+    }
+  }
+
+  /** @return the renewal time in milliseconds. */
+  private synchronized long getRenewalTime() {
+    return renewal;
+  }
+
+  /** Used for testing only. */
+  @VisibleForTesting
+  public synchronized void setRenewalTime(final long renewal) {
+    this.renewal = renewal;
+  }
+
+  /** Add a client. */
+  private synchronized void addClient(final DFSClient dfsc) {
+    for(DFSClient c : dfsclients) {
+      if (c == dfsc) {
+        //client already exists, nothing to do.
+        return;
+      }
+    }
+    //client not found, add it
+    dfsclients.add(dfsc);
+
+    //update renewal time
+    final int hdfsTimeout = dfsc.getConf().getHdfsTimeout();
+    if (hdfsTimeout > 0) {
+      final long half = hdfsTimeout/2;
+      if (half < renewal) {
+        this.renewal = half;
+      }
+    }
+  }
+
+  private synchronized boolean clientsRunning() {
+    for(Iterator<DFSClient> i = dfsclients.iterator(); i.hasNext(); ) {
+      if (!i.next().isClientRunning()) {
+        i.remove();
+      }
+    }
+    return !dfsclients.isEmpty();
+  }
+
+  private synchronized long getSleepPeriod() {
+    return sleepPeriod;
+  }
+
+  /** Set the grace period and adjust the sleep period accordingly. */
+  synchronized void setGraceSleepPeriod(final long gracePeriod) {
+    unsyncSetGraceSleepPeriod(gracePeriod);
+  }
+
+  private void unsyncSetGraceSleepPeriod(final long gracePeriod) {
+    if (gracePeriod < 100L) {
+      throw new HadoopIllegalArgumentException(gracePeriod
+          + " = gracePeriod < 100ms is too small.");
+    }
+    this.gracePeriod = gracePeriod;
+    final long half = gracePeriod/2;
+    this.sleepPeriod = half < LEASE_RENEWER_SLEEP_DEFAULT?
+        half: LEASE_RENEWER_SLEEP_DEFAULT;
+  }
+
+  /** Is the daemon running? */
+  synchronized boolean isRunning() {
+    return daemon != null && daemon.isAlive();
+  }
+
+  /** Does this renewer have nothing to renew? */
+  public boolean isEmpty() {
+    return dfsclients.isEmpty();
+  }
+
+  /** Used only by tests */
+  synchronized String getDaemonName() {
+    return daemon.getName();
+  }
+
+  /** Is the empty period longer than the grace period? */
+  private synchronized boolean isRenewerExpired() {
+    return emptyTime != Long.MAX_VALUE
+        && Time.monotonicNow() - emptyTime > gracePeriod;
+  }
+
+  public synchronized void put(final long inodeId, final DFSOutputStream out,
+      final DFSClient dfsc) {
+    if (dfsc.isClientRunning()) {
+      if (!isRunning() || isRenewerExpired()) {
+        //start a new deamon with a new id.
+        final int id = ++currentId;
+        daemon = new Daemon(new Runnable() {
+          @Override
+          public void run() {
+            try {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Lease renewer daemon for " + clientsString()
+                    + " with renew id " + id + " started");
+              }
+              LeaseRenewer.this.run(id);
+            } catch(InterruptedException e) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
+                    + " is interrupted.", e);
+              }
+            } finally {
+              synchronized(LeaseRenewer.this) {
+                Factory.INSTANCE.remove(LeaseRenewer.this);
+              }
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Lease renewer daemon for " + clientsString()
+                    + " with renew id " + id + " exited");
+              }
+            }
+          }
+
+          @Override
+          public String toString() {
+            return String.valueOf(LeaseRenewer.this);
+          }
+        });
+        daemon.start();
+      }
+      dfsc.putFileBeingWritten(inodeId, out);
+      emptyTime = Long.MAX_VALUE;
+    }
+  }
+
+  @VisibleForTesting
+  synchronized void setEmptyTime(long time) {
+    emptyTime = time;
+  }
+
+  /** Close a file. */
+  public void closeFile(final long inodeId, final DFSClient dfsc) {
+    dfsc.removeFileBeingWritten(inodeId);
+
+    synchronized(this) {
+      if (dfsc.isFilesBeingWrittenEmpty()) {
+        dfsclients.remove(dfsc);
+      }
+      //update emptyTime if necessary
+      if (emptyTime == Long.MAX_VALUE) {
+        for(DFSClient c : dfsclients) {
+          if (!c.isFilesBeingWrittenEmpty()) {
+            //found a non-empty file-being-written map
+            return;
+          }
+        }
+        //discover the first time that all file-being-written maps are empty.
+        emptyTime = Time.monotonicNow();
+      }
+    }
+  }
+
+  /** Close the given client. */
+  public synchronized void closeClient(final DFSClient dfsc) {
+    dfsclients.remove(dfsc);
+    if (dfsclients.isEmpty()) {
+      if (!isRunning() || isRenewerExpired()) {
+        Factory.INSTANCE.remove(LeaseRenewer.this);
+        return;
+      }
+      if (emptyTime == Long.MAX_VALUE) {
+        //discover the first time that the client list is empty.
+        emptyTime = Time.monotonicNow();
+      }
+    }
+
+    //update renewal time
+    if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
+      long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
+      for(DFSClient c : dfsclients) {
+        final int timeout = c.getConf().getHdfsTimeout();
+        if (timeout > 0 && timeout < min) {
+          min = timeout;
+        }
+      }
+      renewal = min/2;
+    }
+  }
+
+  public void interruptAndJoin() throws InterruptedException {
+    Daemon daemonCopy = null;
+    synchronized (this) {
+      if (isRunning()) {
+        daemon.interrupt();
+        daemonCopy = daemon;
+      }
+    }
+
+    if (daemonCopy != null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Wait for lease checker to terminate");
+      }
+      daemonCopy.join();
+    }
+  }
+
+  private void renew() throws IOException {
+    final List<DFSClient> copies;
+    synchronized(this) {
+      copies = new ArrayList<DFSClient>(dfsclients);
+    }
+    //sort the client names for finding out repeated names.
+    Collections.sort(copies, new Comparator<DFSClient>() {
+      @Override
+      public int compare(final DFSClient left, final DFSClient right) {
+        return left.getClientName().compareTo(right.getClientName());
+      }
+    });
+    String previousName = "";
+    for(int i = 0; i < copies.size(); i++) {
+      final DFSClient c = copies.get(i);
+      //skip if current client name is the same as the previous name.
+      if (!c.getClientName().equals(previousName)) {
+        if (!c.renewLease()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Did not renew lease for client " +
+                c);
+          }
+          continue;
+        }
+        previousName = c.getClientName();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Lease renewed for client " + previousName);
+        }
+      }
+    }
+  }
+
+  /**
+   * Periodically check in with the namenode and renew all the leases
+   * when the lease period is half over.
+   */
+  private void run(final int id) throws InterruptedException {
+    for(long lastRenewed = Time.monotonicNow(); !Thread.interrupted();
+        Thread.sleep(getSleepPeriod())) {
+      final long elapsed = Time.monotonicNow() - lastRenewed;
+      if (elapsed >= getRenewalTime()) {
+        try {
+          renew();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Lease renewer daemon for " + clientsString()
+                + " with renew id " + id + " executed");
+          }
+          lastRenewed = Time.monotonicNow();
+        } catch (SocketTimeoutException ie) {
+          LOG.warn("Failed to renew lease for " + clientsString() + " for "
+              + (elapsed/1000) + " seconds.  Aborting ...", ie);
+          synchronized (this) {
+            while (!dfsclients.isEmpty()) {
+              DFSClient dfsClient = dfsclients.get(0);
+              dfsClient.closeAllFilesBeingWritten(true);
+              closeClient(dfsClient);
+            }
+            //Expire the current LeaseRenewer thread.
+            emptyTime = 0;
+          }
+          break;
+        } catch (IOException ie) {
+          LOG.warn("Failed to renew lease for " + clientsString() + " for "
+              + (elapsed/1000) + " seconds.  Will retry shortly ...", ie);
+        }
+      }
+
+      synchronized(this) {
+        if (id != currentId || isRenewerExpired()) {
+          if (LOG.isDebugEnabled()) {
+            if (id != currentId) {
+              LOG.debug("Lease renewer daemon for " + clientsString()
+                  + " with renew id " + id + " is not current");
+            } else {
+               LOG.debug("Lease renewer daemon for " + clientsString()
+                  + " with renew id " + id + " expired");
+            }
+          }
+          //no longer the current daemon or expired
+          return;
+        }
+
+        // if no clients are in running state or there is no more clients
+        // registered with this renewer, stop the daemon after the grace
+        // period.
+        if (!clientsRunning() && emptyTime == Long.MAX_VALUE) {
+          emptyTime = Time.monotonicNow();
+        }
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    String s = getClass().getSimpleName() + ":" + factorykey;
+    if (LOG.isTraceEnabled()) {
+      return s + ", clients=" +  clientsString()
+        + ", created at " + instantiationTrace;
+    }
+    return s;
+  }
+
+  /** Get the names of all clients */
+  private synchronized String clientsString() {
+    if (dfsclients.isEmpty()) {
+      return "[]";
+    } else {
+      final StringBuilder b = new StringBuilder("[").append(
+          dfsclients.get(0).getClientName());
+      for(int i = 1; i < dfsclients.size(); i++) {
+        b.append(", ").append(dfsclients.get(i).getClientName());
+      }
+      return b.append("]").toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
new file mode 100644
index 0000000..e4b51c5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.inotify;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class MissingEventsException extends Exception {
+  private static final long serialVersionUID = 1L;
+
+  private long expectedTxid;
+  private long actualTxid;
+
+  public MissingEventsException() {}
+
+  public MissingEventsException(long expectedTxid, long actualTxid) {
+    this.expectedTxid = expectedTxid;
+    this.actualTxid = actualTxid;
+  }
+
+  public long getExpectedTxid() {
+    return expectedTxid;
+  }
+
+  public long getActualTxid() {
+    return actualTxid;
+  }
+
+  @Override
+  public String toString() {
+    return "We expected the next batch of events to start with transaction ID "
+        + expectedTxid + ", but it instead started with transaction ID " +
+        actualTxid + ". Most likely the intervening transactions were cleaned "
+        + "up as part of checkpointing.";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
new file mode 100644
index 0000000..1210999
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Indicates a failure manipulating an ACL.
+ */
+@InterfaceAudience.Private
+public class AclException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * Creates a new AclException.
+   *
+   * @param message String message
+   */
+  public AclException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
new file mode 100644
index 0000000..923cdb4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * CacheDirectiveIterator is a remote iterator that iterates cache directives.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CacheDirectiveIterator
+    extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
+
+  private CacheDirectiveInfo filter;
+  private final ClientProtocol namenode;
+  private final Sampler<?> traceSampler;
+
+  public CacheDirectiveIterator(ClientProtocol namenode,
+      CacheDirectiveInfo filter, Sampler<?> traceSampler) {
+    super(0L);
+    this.namenode = namenode;
+    this.filter = filter;
+    this.traceSampler = traceSampler;
+  }
+
+  private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
+    CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder(filter);
+    builder.setId(null);
+    return builder.build();
+  }
+
+  /**
+   * Used for compatibility when communicating with a server version that
+   * does not support filtering directives by ID.
+   */
+  private static class SingleEntry implements
+      BatchedEntries<CacheDirectiveEntry> {
+
+    private final CacheDirectiveEntry entry;
+
+    public SingleEntry(final CacheDirectiveEntry entry) {
+      this.entry = entry;
+    }
+
+    @Override
+    public CacheDirectiveEntry get(int i) {
+      if (i > 0) {
+        return null;
+      }
+      return entry;
+    }
+
+    @Override
+    public int size() {
+      return 1;
+    }
+
+    @Override
+    public boolean hasMore() {
+      return false;
+    }
+  }
+
+  @Override
+  public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
+      throws IOException {
+    BatchedEntries<CacheDirectiveEntry> entries = null;
+    TraceScope scope = Trace.startSpan("listCacheDirectives", traceSampler);
+    try {
+      entries = namenode.listCacheDirectives(prevKey, filter);
+    } catch (IOException e) {
+      if (e.getMessage().contains("Filtering by ID is unsupported")) {
+        // Retry case for old servers, do the filtering client-side
+        long id = filter.getId();
+        filter = removeIdFromFilter(filter);
+        // Using id - 1 as prevId should get us a window containing the id
+        // This is somewhat brittle, since it depends on directives being
+        // returned in order of ascending ID.
+        entries = namenode.listCacheDirectives(id - 1, filter);
+        for (int i=0; i<entries.size(); i++) {
+          CacheDirectiveEntry entry = entries.get(i);
+          if (entry.getInfo().getId().equals((Long)id)) {
+            return new SingleEntry(entry);
+          }
+        }
+        throw new RemoteException(InvalidRequestException.class.getName(),
+            "Did not find requested id " + id);
+      }
+      throw e;
+    } finally {
+      scope.close();
+    }
+    Preconditions.checkNotNull(entries);
+    return entries;
+  }
+
+  @Override
+  public Long elementToPrevKey(CacheDirectiveEntry entry) {
+    return entry.getInfo().getId();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
new file mode 100644
index 0000000..e9481f7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * CachePoolIterator is a remote iterator that iterates cache pools.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CachePoolIterator
+    extends BatchedRemoteIterator<String, CachePoolEntry> {
+
+  private final ClientProtocol namenode;
+  private final Sampler traceSampler;
+
+  public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) {
+    super("");
+    this.namenode = namenode;
+    this.traceSampler = traceSampler;
+  }
+
+  @Override
+  public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
+      throws IOException {
+    TraceScope scope = Trace.startSpan("listCachePools", traceSampler);
+    try {
+      return namenode.listCachePools(prevKey);
+    } finally {
+      scope.close();
+    }
+  }
+
+  @Override
+  public String elementToPrevKey(CachePoolEntry entry) {
+    return entry.getInfo().getPoolName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
new file mode 100644
index 0000000..0141215
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * EncryptionZoneIterator is a remote iterator that iterates over encryption
+ * zones. It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class EncryptionZoneIterator
+    extends BatchedRemoteIterator<Long, EncryptionZone> {
+
+  private final ClientProtocol namenode;
+  private final Sampler<?> traceSampler;
+
+  public EncryptionZoneIterator(ClientProtocol namenode,
+                                Sampler<?> traceSampler) {
+    super(Long.valueOf(0));
+    this.namenode = namenode;
+    this.traceSampler = traceSampler;
+  }
+
+  @Override
+  public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
+      throws IOException {
+    TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler);
+    try {
+      return namenode.listEncryptionZones(prevId);
+    } finally {
+      scope.close();
+    }
+  }
+
+  @Override
+  public Long elementToPrevKey(EncryptionZone entry) {
+    return entry.getId();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
new file mode 100644
index 0000000..25084c7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
+
+import static org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.long2String;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class QuotaByStorageTypeExceededException extends QuotaExceededException {
+  protected static final long serialVersionUID = 1L;
+  protected StorageType type;
+
+  public QuotaByStorageTypeExceededException() {}
+
+  public QuotaByStorageTypeExceededException(String msg) {
+    super(msg);
+  }
+
+  public QuotaByStorageTypeExceededException(long quota, long count, StorageType type) {
+    super(quota, count);
+    this.type = type;
+  }
+
+  @Override
+  public String getMessage() {
+    String msg = super.getMessage();
+    if (msg == null) {
+      return "Quota by storage type : " + type.toString() +
+          " on path : " + (pathName==null ? "": pathName) +
+          " is exceeded. quota = "  + long2String(quota, "B", 2) +
+          " but space consumed = " + long2String(count, "B", 2);
+    } else {
+      return msg;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
new file mode 100644
index 0000000..03fb704
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.Path;
+
+/** 
+ * Thrown when a symbolic link is encountered in a path.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class UnresolvedPathException extends UnresolvedLinkException {
+  private static final long serialVersionUID = 1L;
+  private String path;        // The path containing the link
+  private String preceding;   // The path part preceding the link
+  private String remainder;   // The path part following the link
+  private String linkTarget;  // The link's target
+
+  /**
+   * Used by RemoteException to instantiate an UnresolvedPathException.
+   */
+  public UnresolvedPathException(String msg) {
+    super(msg);
+  }
+  
+  public UnresolvedPathException(String path, String preceding,
+      String remainder, String linkTarget) {
+    this.path = path;
+    this.preceding = preceding;
+    this.remainder = remainder;
+    this.linkTarget = linkTarget;
+  }
+
+  /**
+   * Return a path with the link resolved with the target.
+   */
+  public Path getResolvedPath() throws IOException {
+    // If the path is absolute we cam throw out the preceding part and
+    // just append the remainder to the target, otherwise append each
+    // piece to resolve the link in path.
+    boolean noRemainder = (remainder == null || "".equals(remainder));
+    Path target = new Path(linkTarget);
+    if (target.isUriPathAbsolute()) {
+      return noRemainder ? target : new Path(target, remainder);
+    } else {
+      return noRemainder
+        ? new Path(preceding, target)
+        : new Path(new Path(preceding, linkTarget), remainder);
+    }
+  }
+
+  @Override
+  public String getMessage() {
+    String msg = super.getMessage();
+    if (msg != null) {
+      return msg;
+    }
+    String myMsg = "Unresolved path " + path;
+    try {
+      return getResolvedPath().toString();
+    } catch (IOException e) {
+      // Ignore
+    }
+    return myMsg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
new file mode 100644
index 0000000..c69986a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol.datatransfer;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+
+/**
+ * The setting of replace-datanode-on-failure feature.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ReplaceDatanodeOnFailure {
+  /** The replacement policies */
+  public enum Policy {
+    /** The feature is disabled in the entire site. */
+    DISABLE(Condition.FALSE),
+    /** Never add a new datanode. */
+    NEVER(Condition.FALSE),
+    /** @see ReplaceDatanodeOnFailure.Condition#DEFAULT */
+    DEFAULT(Condition.DEFAULT),
+    /** Always add a new datanode when an existing datanode is removed. */
+    ALWAYS(Condition.TRUE);
+
+    private final Condition condition;
+
+    private Policy(Condition condition) {
+      this.condition = condition;
+    }
+    
+    Condition getCondition() {
+      return condition;
+    }
+  }
+
+  /** Datanode replacement condition */
+  private static interface Condition {
+    /** Return true unconditionally. */
+    static final Condition TRUE = new Condition() {
+      @Override
+      public boolean satisfy(short replication, DatanodeInfo[] existings,
+          int nExistings, boolean isAppend, boolean isHflushed) {
+        return true;
+      }
+    };
+
+    /** Return false unconditionally. */
+    static final Condition FALSE = new Condition() {
+      @Override
+      public boolean satisfy(short replication, DatanodeInfo[] existings,
+          int nExistings, boolean isAppend, boolean isHflushed) {
+        return false;
+      }
+    };
+
+    /**
+     * DEFAULT condition:
+     *   Let r be the replication number.
+     *   Let n be the number of existing datanodes.
+     *   Add a new datanode only if r >= 3 and either
+     *   (1) floor(r/2) >= n; or
+     *   (2) r > n and the block is hflushed/appended.
+     */
+    static final Condition DEFAULT = new Condition() {
+      @Override
+      public boolean satisfy(final short replication,
+          final DatanodeInfo[] existings, final int n, final boolean isAppend,
+          final boolean isHflushed) {
+        if (replication < 3) {
+          return false;
+        } else {
+          if (n <= (replication/2)) {
+            return true;
+          } else {
+            return isAppend || isHflushed;
+          }
+        }
+      }
+    };
+
+    /** Is the condition satisfied? */
+    public boolean satisfy(short replication, DatanodeInfo[] existings,
+        int nExistings, boolean isAppend, boolean isHflushed);
+  }
+
+  private final Policy policy;
+  private final boolean bestEffort;
+  
+  public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
+    this.policy = policy;
+    this.bestEffort = bestEffort;
+  }
+
+  /** Check if the feature is enabled. */
+  public void checkEnabled() {
+    if (policy == Policy.DISABLE) {
+      throw new UnsupportedOperationException(
+          "This feature is disabled.  Please refer to "
+          + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY
+          + " configuration property.");
+    }
+  }
+
+  /**
+   * Best effort means that the client will try to replace the failed datanode
+   * (provided that the policy is satisfied), however, it will continue the
+   * write operation in case that the datanode replacement also fails.
+   * 
+   * @return Suppose the datanode replacement fails.
+   *     false: An exception should be thrown so that the write will fail.
+   *     true : The write should be resumed with the remaining datandoes.
+   */
+  public boolean isBestEffort() {
+    return bestEffort;
+  }
+
+  /** Does it need a replacement according to the policy? */
+  public boolean satisfy(
+      final short replication, final DatanodeInfo[] existings,
+      final boolean isAppend, final boolean isHflushed) {
+    final int n = existings == null? 0: existings.length;
+    if (n == 0 || n >= replication) {
+      //don't need to add datanode for any policy.
+      return false;
+    } else {
+      return policy.getCondition().satisfy(
+          replication, existings, n, isAppend, isHflushed);
+    }
+  }
+  
+  @Override
+  public String toString() {
+    return policy.toString();
+  }
+
+  /** Get the setting from configuration. */
+  public static ReplaceDatanodeOnFailure get(final Configuration conf) {
+    final Policy policy = getPolicy(conf);
+    final boolean bestEffort = conf.getBoolean(
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
+    
+    return new ReplaceDatanodeOnFailure(policy, bestEffort);
+  }
+
+  private static Policy getPolicy(final Configuration conf) {
+    final boolean enabled = conf.getBoolean(
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_DEFAULT);
+    if (!enabled) {
+      return Policy.DISABLE;
+    }
+
+    final String policy = conf.get(
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_DEFAULT);
+    for(int i = 1; i < Policy.values().length; i++) {
+      final Policy p = Policy.values()[i];
+      if (p.name().equalsIgnoreCase(policy)) {
+        return p;
+      }
+    }
+    throw new HadoopIllegalArgumentException("Illegal configuration value for "
+        + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
+        + ": " + policy);
+  }
+
+  /** Write the setting to configuration. */
+  public static void write(final Policy policy,
+      final boolean bestEffort, final Configuration conf) {
+    conf.setBoolean(
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
+        policy != Policy.DISABLE);
+    conf.set(
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
+        policy.name());
+    conf.setBoolean(
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
+        bestEffort);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
new file mode 100644
index 0000000..b159d3a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+
+/**
+ * Exception indicating that DataNode does not have a replica
+ * that matches the target block.  
+ */
+public class ReplicaNotFoundException extends IOException {
+  private static final long serialVersionUID = 1L;
+  public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
+  public final static String UNFINALIZED_REPLICA = 
+    "Cannot append to an unfinalized replica ";
+  public final static String UNFINALIZED_AND_NONRBW_REPLICA = 
+    "Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
+  public final static String NON_EXISTENT_REPLICA =
+    "Cannot append to a non-existent replica ";
+  public final static String UNEXPECTED_GS_REPLICA =
+    "Cannot append to a replica with unexpected generation stamp ";
+
+  public ReplicaNotFoundException() {
+    super();
+  }
+
+  public ReplicaNotFoundException(ExtendedBlock b) {
+    super("Replica not found for " + b);
+  }
+  
+  public ReplicaNotFoundException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
new file mode 100644
index 0000000..0bdd2a5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class RetryStartFileException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public RetryStartFileException() {
+    super("Preconditions for creating a file failed because of a " +
+        "transient error, retry create later.");
+  }
+
+  public RetryStartFileException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c3a1668..38af55b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -629,6 +629,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8873. Allow the directoryScanner to be rate-limited (Daniel Templeton
     via Colin P. McCabe)
 
+    HDFS-8053. Move DFSIn/OutputStream and related classes to
+    hadoop-hdfs-client. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 60029e0..c88c4c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -74,15 +74,6 @@
      </Match>
 
      <!--
-      ResponseProccessor is thread that is designed to catch RuntimeException.
-     -->
-     <Match>
-       <Class name="org.apache.hadoop.hdfs.DataStreamer$ResponseProcessor" />
-       <Method name="run" />
-       <Bug pattern="REC_CATCH_EXCEPTION" />
-     </Match>
-
-     <!--
       lastAppliedTxid is carefully unsynchronized in the BackupNode in a couple spots.
       See the comments in BackupImage for justification.
      -->
@@ -196,14 +187,4 @@
       <Method name="assertAllResultsEqual" />
       <Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE" />
     </Match>
-
-    <!--
-     We use a separate lock to guard cachingStrategy in order to separate
-     locks for p-reads from seek + read invocations.
-    -->
-    <Match>
-        <Class name="org.apache.hadoop.hdfs.DFSInputStream" />
-        <Field name="cachingStrategy" />
-        <Bug pattern="IS2_INCONSISTENT_SYNC" />
-    </Match>
  </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
deleted file mode 100644
index 2200994..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Wrapper for {@link BlockLocation} that also adds {@link VolumeId} volume
- * location information for each replica.
- */
-@InterfaceStability.Unstable
-@InterfaceAudience.Public
-@Deprecated
-public class BlockStorageLocation extends BlockLocation {
-
-  private final VolumeId[] volumeIds;
-
-  public BlockStorageLocation(BlockLocation loc, VolumeId[] volumeIds)
-      throws IOException {
-    // Initialize with data from passed in BlockLocation
-    super(loc.getNames(), loc.getHosts(), loc.getTopologyPaths(), loc
-        .getOffset(), loc.getLength(), loc.isCorrupt());
-    this.volumeIds = volumeIds;
-  }
-
-  /**
-   * Gets the list of {@link VolumeId} corresponding to the block's replicas.
-   * 
-   * @return volumeIds list of VolumeId for the block's replicas
-   */
-  public VolumeId[] getVolumeIds() {
-    return volumeIds;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
deleted file mode 100644
index 0ccacda..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-
-/**
- * Wrapper for {@link BlockLocation} that also includes a {@link LocatedBlock},
- * allowing more detailed queries to the datanode about a block.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class HdfsBlockLocation extends BlockLocation {
-
-  private final LocatedBlock block;
-  
-  public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) 
-      throws IOException {
-    // Initialize with data from passed in BlockLocation
-    super(loc);
-    this.block = block;
-  }
-  
-  public LocatedBlock getLocatedBlock() {
-    return block;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
deleted file mode 100644
index 6e9d3d7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.base.Preconditions;
-
-/**
- * HDFS-specific volume identifier which implements {@link VolumeId}. Can be
- * used to differentiate between the data directories on a single datanode. This
- * identifier is only unique on a per-datanode basis.
- */
-@InterfaceStability.Unstable
-@InterfaceAudience.Public
-public class HdfsVolumeId implements VolumeId {
-  
-  private final byte[] id;
-
-  public HdfsVolumeId(byte[] id) {
-    Preconditions.checkNotNull(id, "id cannot be null");
-    this.id = id;
-  }
-
-  @Override
-  public int compareTo(VolumeId arg0) {
-    if (arg0 == null) {
-      return 1;
-    }
-    return hashCode() - arg0.hashCode();
-  }
-
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder().append(id).toHashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null || obj.getClass() != getClass()) {
-      return false;
-    }
-    if (obj == this) {
-      return true;
-    }
-    HdfsVolumeId that = (HdfsVolumeId) obj;
-    return new EqualsBuilder().append(this.id, that.id).isEquals();
-  }
-
-  @Override
-  public String toString() {
-    return StringUtils.byteToHexString(id);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
deleted file mode 100644
index e56e304..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Opaque interface that identifies a disk location. Subclasses
- * should implement {@link Comparable} and override both equals and hashCode.
- */
-@InterfaceStability.Unstable
-@InterfaceAudience.Public
-public interface VolumeId extends Comparable<VolumeId> {
-
-  @Override
-  abstract public int compareTo(VolumeId arg0);
-
-  @Override
-  abstract public int hashCode();
-
-  @Override
-  abstract public boolean equals(Object obj);
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
deleted file mode 100644
index 7bba8a4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/** 
-  * This exception is thrown when a read encounters a block that has no locations
-  * associated with it.
-  */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BlockMissingException extends IOException {
-
-  private static final long serialVersionUID = 1L;
-
-  private final String filename;
-  private final long   offset;
-
-  /**
-   * An exception that indicates that file was corrupted.
-   * @param filename name of corrupted file
-   * @param description a description of the corruption details
-   */
-  public BlockMissingException(String filename, String description, long offset) {
-    super(description);
-    this.filename = filename;
-    this.offset = offset;
-  }
-
-  /**
-   * Returns the name of the corrupted file.
-   * @return name of corrupted file
-   */
-  public String getFile() {
-    return filename;
-  }
-
-  /**
-   * Returns the offset at which this file is corrupted
-   * @return offset of corrupted file
-   */
-  public long getOffset() {
-    return offset;
-  }
-}


[04/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
deleted file mode 100644
index 5392c66..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Used for injecting faults in DFSClient and DFSOutputStream tests.
- * Calls into this are a no-op in production code. 
- */
-@VisibleForTesting
-@InterfaceAudience.Private
-public class DFSClientFaultInjector {
-  public static DFSClientFaultInjector instance = new DFSClientFaultInjector();
-  public static AtomicLong exceptionNum = new AtomicLong(0);
-
-  public static DFSClientFaultInjector get() {
-    return instance;
-  }
-
-  public boolean corruptPacket() {
-    return false;
-  }
-
-  public boolean uncorruptPacket() {
-    return false;
-  }
-
-  public boolean failPacket() {
-    return false;
-  }
-
-  public void startFetchFromDatanode() {}
-
-  public void fetchFromDatanodeException() {}
-
-  public void readFromDatanodeDelay() {}
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index c9f23f9..0ba174f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hdfs;
 
-import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
deleted file mode 100644
index 2a228e8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * The client-side metrics for hedged read feature.
- * This class has a number of metrics variables that are publicly accessible,
- * we can grab them from client side, like HBase.
- */
-@InterfaceAudience.Private
-public class DFSHedgedReadMetrics {
-  public final AtomicLong hedgedReadOps = new AtomicLong();
-  public final AtomicLong hedgedReadOpsWin = new AtomicLong();
-  public final AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
-
-  public void incHedgedReadOps() {
-    hedgedReadOps.incrementAndGet();
-  }
-
-  public void incHedgedReadOpsInCurThread() {
-    hedgedReadOpsInCurThread.incrementAndGet();
-  }
-
-  public void incHedgedReadWins() {
-    hedgedReadOpsWin.incrementAndGet();
-  }
-
-  public long getHedgedReadOps() {
-    return hedgedReadOps.longValue();
-  }
-
-  public long getHedgedReadOpsInCurThread() {
-    return hedgedReadOpsInCurThread.longValue();
-  }
-
-  public long getHedgedReadWins() {
-    return hedgedReadOpsWin.longValue();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
deleted file mode 100644
index 1f9e3e9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import com.google.common.collect.Iterators;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.inotify.EventBatch;
-import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.inotify.MissingEventsException;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Stream for reading inotify events. DFSInotifyEventInputStreams should not
- * be shared among multiple threads.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public class DFSInotifyEventInputStream {
-  public static Logger LOG = LoggerFactory.getLogger(DFSInotifyEventInputStream
-      .class);
-
-  /**
-   * The trace sampler to use when making RPCs to the NameNode.
-   */
-  private final Sampler<?> traceSampler;
-
-  private final ClientProtocol namenode;
-  private Iterator<EventBatch> it;
-  private long lastReadTxid;
-  /**
-   * The most recent txid the NameNode told us it has sync'ed -- helps us
-   * determine how far behind we are in the edit stream.
-   */
-  private long syncTxid;
-  /**
-   * Used to generate wait times in {@link DFSInotifyEventInputStream#take()}.
-   */
-  private Random rng = new Random();
-
-  private static final int INITIAL_WAIT_MS = 10;
-
-  DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode)
-        throws IOException {
-    // Only consider new transaction IDs.
-    this(traceSampler, namenode, namenode.getCurrentEditLogTxid());
-  }
-
-  DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode,
-        long lastReadTxid) throws IOException {
-    this.traceSampler = traceSampler;
-    this.namenode = namenode;
-    this.it = Iterators.emptyIterator();
-    this.lastReadTxid = lastReadTxid;
-  }
-
-  /**
-   * Returns the next batch of events in the stream or null if no new
-   * batches are currently available.
-   *
-   * @throws IOException because of network error or edit log
-   * corruption. Also possible if JournalNodes are unresponsive in the
-   * QJM setting (even one unresponsive JournalNode is enough in rare cases),
-   * so catching this exception and retrying at least a few times is
-   * recommended.
-   * @throws MissingEventsException if we cannot return the next batch in the
-   * stream because the data for the events (and possibly some subsequent
-   * events) has been deleted (generally because this stream is a very large
-   * number of transactions behind the current state of the NameNode). It is
-   * safe to continue reading from the stream after this exception is thrown
-   * The next available batch of events will be returned.
-   */
-  public EventBatch poll() throws IOException, MissingEventsException {
-    TraceScope scope =
-        Trace.startSpan("inotifyPoll", traceSampler);
-    try {
-      // need to keep retrying until the NN sends us the latest committed txid
-      if (lastReadTxid == -1) {
-        LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
-        lastReadTxid = namenode.getCurrentEditLogTxid();
-        return null;
-      }
-      if (!it.hasNext()) {
-        EventBatchList el = namenode.getEditsFromTxid(lastReadTxid + 1);
-        if (el.getLastTxid() != -1) {
-          // we only want to set syncTxid when we were actually able to read some
-          // edits on the NN -- otherwise it will seem like edits are being
-          // generated faster than we can read them when the problem is really
-          // that we are temporarily unable to read edits
-          syncTxid = el.getSyncTxid();
-          it = el.getBatches().iterator();
-          long formerLastReadTxid = lastReadTxid;
-          lastReadTxid = el.getLastTxid();
-          if (el.getFirstTxid() != formerLastReadTxid + 1) {
-            throw new MissingEventsException(formerLastReadTxid + 1,
-                el.getFirstTxid());
-          }
-        } else {
-          LOG.debug("poll(): read no edits from the NN when requesting edits " +
-            "after txid {}", lastReadTxid);
-          return null;
-        }
-      }
-
-      if (it.hasNext()) { // can be empty if el.getLastTxid != -1 but none of the
-        // newly seen edit log ops actually got converted to events
-        return it.next();
-      } else {
-        return null;
-      }
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Return a estimate of how many transaction IDs behind the NameNode's
-   * current state this stream is. Clients should periodically call this method
-   * and check if its result is steadily increasing, which indicates that they
-   * are falling behind (i.e. transaction are being generated faster than the
-   * client is reading them). If a client falls too far behind events may be
-   * deleted before the client can read them.
-   * <p/>
-   * A return value of -1 indicates that an estimate could not be produced, and
-   * should be ignored. The value returned by this method is really only useful
-   * when compared to previous or subsequent returned values.
-   */
-  public long getTxidsBehindEstimate() {
-    if (syncTxid == 0) {
-      return -1;
-    } else {
-      assert syncTxid >= lastReadTxid;
-      // this gives the difference between the last txid we have fetched to the
-      // client and syncTxid at the time we last fetched events from the
-      // NameNode
-      return syncTxid - lastReadTxid;
-    }
-  }
-
-  /**
-   * Returns the next event batch in the stream, waiting up to the specified
-   * amount of time for a new batch. Returns null if one is not available at the
-   * end of the specified amount of time. The time before the method returns may
-   * exceed the specified amount of time by up to the time required for an RPC
-   * to the NameNode.
-   *
-   * @param time number of units of the given TimeUnit to wait
-   * @param tu the desired TimeUnit
-   * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
-   * @throws MissingEventsException
-   * see {@link DFSInotifyEventInputStream#poll()}
-   * @throws InterruptedException if the calling thread is interrupted
-   */
-  public EventBatch poll(long time, TimeUnit tu) throws IOException,
-      InterruptedException, MissingEventsException {
-    TraceScope scope = Trace.startSpan("inotifyPollWithTimeout", traceSampler);
-    EventBatch next = null;
-    try {
-      long initialTime = Time.monotonicNow();
-      long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
-      long nextWait = INITIAL_WAIT_MS;
-      while ((next = poll()) == null) {
-        long timeLeft = totalWait - (Time.monotonicNow() - initialTime);
-        if (timeLeft <= 0) {
-          LOG.debug("timed poll(): timed out");
-          break;
-        } else if (timeLeft < nextWait * 2) {
-          nextWait = timeLeft;
-        } else {
-          nextWait *= 2;
-        }
-        LOG.debug("timed poll(): poll() returned null, sleeping for {} ms",
-            nextWait);
-        Thread.sleep(nextWait);
-      }
-    } finally {
-      scope.close();
-    }
-    return next;
-  }
-
-  /**
-   * Returns the next batch of events in the stream, waiting indefinitely if
-   * a new batch  is not immediately available.
-   *
-   * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
-   * @throws MissingEventsException see
-   * {@link DFSInotifyEventInputStream#poll()}
-   * @throws InterruptedException if the calling thread is interrupted
-   */
-  public EventBatch take() throws IOException, InterruptedException,
-      MissingEventsException {
-    TraceScope scope = Trace.startSpan("inotifyTake", traceSampler);
-    EventBatch next = null;
-    try {
-      int nextWaitMin = INITIAL_WAIT_MS;
-      while ((next = poll()) == null) {
-        // sleep for a random period between nextWaitMin and nextWaitMin * 2
-        // to avoid stampedes at the NN if there are multiple clients
-        int sleepTime = nextWaitMin + rng.nextInt(nextWaitMin);
-        LOG.debug("take(): poll() returned null, sleeping for {} ms", sleepTime);
-        Thread.sleep(sleepTime);
-        // the maximum sleep is 2 minutes
-        nextWaitMin = Math.min(60000, nextWaitMin * 2);
-      }
-    } finally {
-      scope.close();
-    }
-
-    return next;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
deleted file mode 100644
index 139a27c..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ /dev/null
@@ -1,1915 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.fs.ByteBufferUtil;
-import org.apache.hadoop.fs.CanSetDropBehind;
-import org.apache.hadoop.fs.CanSetReadahead;
-import org.apache.hadoop.fs.CanUnbuffer;
-import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.FSInputStream;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
-import org.apache.hadoop.fs.ReadOption;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
-import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
-import org.apache.hadoop.io.ByteBufferPool;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.IdentityHashStore;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/****************************************************************
- * DFSInputStream provides bytes from a named file.  It handles 
- * negotiation of the namenode and various datanodes as necessary.
- ****************************************************************/
-@InterfaceAudience.Private
-public class DFSInputStream extends FSInputStream
-implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
-    HasEnhancedByteBufferAccess, CanUnbuffer {
-  @VisibleForTesting
-  public static boolean tcpReadsDisabledForTesting = false;
-  private long hedgedReadOpsLoopNumForTesting = 0;
-  protected final DFSClient dfsClient;
-  protected AtomicBoolean closed = new AtomicBoolean(false);
-  protected final String src;
-  protected final boolean verifyChecksum;
-
-  // state by stateful read only:
-  // (protected by lock on this)
-  /////
-  private DatanodeInfo currentNode = null;
-  protected LocatedBlock currentLocatedBlock = null;
-  protected long pos = 0;
-  protected long blockEnd = -1;
-  private BlockReader blockReader = null;
-  ////
-
-  // state shared by stateful and positional read:
-  // (protected by lock on infoLock)
-  ////
-  protected LocatedBlocks locatedBlocks = null;
-  private long lastBlockBeingWrittenLength = 0;
-  private FileEncryptionInfo fileEncryptionInfo = null;
-  protected CachingStrategy cachingStrategy;
-  ////
-
-  protected final ReadStatistics readStatistics = new ReadStatistics();
-  // lock for state shared between read and pread
-  // Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
-  //       (it's OK to acquire this lock when the lock on <this> is held)
-  protected final Object infoLock = new Object();
-
-  /**
-   * Track the ByteBuffers that we have handed out to readers.
-   * 
-   * The value type can be either ByteBufferPool or ClientMmap, depending on
-   * whether we this is a memory-mapped buffer or not.
-   */
-  private IdentityHashStore<ByteBuffer, Object> extendedReadBuffers;
-
-  private synchronized IdentityHashStore<ByteBuffer, Object>
-        getExtendedReadBuffers() {
-    if (extendedReadBuffers == null) {
-      extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
-    }
-    return extendedReadBuffers;
-  }
-
-  public static class ReadStatistics {
-    public ReadStatistics() {
-      clear();
-    }
-
-    public ReadStatistics(ReadStatistics rhs) {
-      this.totalBytesRead = rhs.getTotalBytesRead();
-      this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
-      this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
-      this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
-    }
-
-    /**
-     * @return The total bytes read.  This will always be at least as
-     * high as the other numbers, since it includes all of them.
-     */
-    public long getTotalBytesRead() {
-      return totalBytesRead;
-    }
-
-    /**
-     * @return The total local bytes read.  This will always be at least
-     * as high as totalShortCircuitBytesRead, since all short-circuit
-     * reads are also local.
-     */
-    public long getTotalLocalBytesRead() {
-      return totalLocalBytesRead;
-    }
-
-    /**
-     * @return The total short-circuit local bytes read.
-     */
-    public long getTotalShortCircuitBytesRead() {
-      return totalShortCircuitBytesRead;
-    }
-    
-    /**
-     * @return The total number of zero-copy bytes read.
-     */
-    public long getTotalZeroCopyBytesRead() {
-      return totalZeroCopyBytesRead;
-    }
-
-    /**
-     * @return The total number of bytes read which were not local.
-     */
-    public long getRemoteBytesRead() {
-      return totalBytesRead - totalLocalBytesRead;
-    }
-    
-    void addRemoteBytes(long amt) {
-      this.totalBytesRead += amt;
-    }
-
-    void addLocalBytes(long amt) {
-      this.totalBytesRead += amt;
-      this.totalLocalBytesRead += amt;
-    }
-
-    void addShortCircuitBytes(long amt) {
-      this.totalBytesRead += amt;
-      this.totalLocalBytesRead += amt;
-      this.totalShortCircuitBytesRead += amt;
-    }
-
-    void addZeroCopyBytes(long amt) {
-      this.totalBytesRead += amt;
-      this.totalLocalBytesRead += amt;
-      this.totalShortCircuitBytesRead += amt;
-      this.totalZeroCopyBytesRead += amt;
-    }
-
-    void clear() {
-      this.totalBytesRead = 0;
-      this.totalLocalBytesRead = 0;
-      this.totalShortCircuitBytesRead = 0;
-      this.totalZeroCopyBytesRead = 0;
-    }
-    
-    private long totalBytesRead;
-
-    private long totalLocalBytesRead;
-
-    private long totalShortCircuitBytesRead;
-
-    private long totalZeroCopyBytesRead;
-  }
-  
-  /**
-   * This variable tracks the number of failures since the start of the
-   * most recent user-facing operation. That is to say, it should be reset
-   * whenever the user makes a call on this stream, and if at any point
-   * during the retry logic, the failure count exceeds a threshold,
-   * the errors will be thrown back to the operation.
-   *
-   * Specifically this counts the number of times the client has gone
-   * back to the namenode to get a new list of block locations, and is
-   * capped at maxBlockAcquireFailures
-   */
-  protected int failures = 0;
-
-  /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
-   * parallel accesses to DFSInputStream (through ptreads) properly */
-  private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
-             new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
-
-  private byte[] oneByteBuf; // used for 'int read()'
-
-  void addToDeadNodes(DatanodeInfo dnInfo) {
-    deadNodes.put(dnInfo, dnInfo);
-  }
-  
-  DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
-    this.dfsClient = dfsClient;
-    this.verifyChecksum = verifyChecksum;
-    this.src = src;
-    synchronized (infoLock) {
-      this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
-    }
-    this.locatedBlocks = locatedBlocks;
-    openInfo(false);
-  }
-
-  /**
-   * Grab the open-file info from namenode
-   * @param refreshLocatedBlocks whether to re-fetch locatedblocks
-   */
-  void openInfo(boolean refreshLocatedBlocks) throws IOException,
-      UnresolvedLinkException {
-    final DfsClientConf conf = dfsClient.getConf();
-    synchronized(infoLock) {
-      lastBlockBeingWrittenLength =
-          fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
-      int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
-      while (retriesForLastBlockLength > 0) {
-        // Getting last block length as -1 is a special case. When cluster
-        // restarts, DNs may not report immediately. At this time partial block
-        // locations will not be available with NN for getting the length. Lets
-        // retry for 3 times to get the length.
-        if (lastBlockBeingWrittenLength == -1) {
-          DFSClient.LOG.warn("Last block locations not available. "
-              + "Datanodes might not have reported blocks completely."
-              + " Will retry for " + retriesForLastBlockLength + " times");
-          waitFor(conf.getRetryIntervalForGetLastBlockLength());
-          lastBlockBeingWrittenLength =
-              fetchLocatedBlocksAndGetLastBlockLength(true);
-        } else {
-          break;
-        }
-        retriesForLastBlockLength--;
-      }
-      if (retriesForLastBlockLength == 0) {
-        throw new IOException("Could not obtain the last block locations.");
-      }
-    }
-  }
-
-  private void waitFor(int waitTime) throws IOException {
-    try {
-      Thread.sleep(waitTime);
-    } catch (InterruptedException e) {
-      throw new IOException(
-          "Interrupted while getting the last block length.");
-    }
-  }
-
-  private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
-      throws IOException {
-    LocatedBlocks newInfo = locatedBlocks;
-    if (locatedBlocks == null || refresh) {
-      newInfo = dfsClient.getLocatedBlocks(src, 0);
-    }
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("newInfo = " + newInfo);
-    }
-    if (newInfo == null) {
-      throw new IOException("Cannot open filename " + src);
-    }
-
-    if (locatedBlocks != null) {
-      Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
-      Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
-      while (oldIter.hasNext() && newIter.hasNext()) {
-        if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
-          throw new IOException("Blocklist for " + src + " has changed!");
-        }
-      }
-    }
-    locatedBlocks = newInfo;
-    long lastBlockBeingWrittenLength = 0;
-    if (!locatedBlocks.isLastBlockComplete()) {
-      final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
-      if (last != null) {
-        if (last.getLocations().length == 0) {
-          if (last.getBlockSize() == 0) {
-            // if the length is zero, then no data has been written to
-            // datanode. So no need to wait for the locations.
-            return 0;
-          }
-          return -1;
-        }
-        final long len = readBlockLength(last);
-        last.getBlock().setNumBytes(len);
-        lastBlockBeingWrittenLength = len; 
-      }
-    }
-
-    fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
-
-    return lastBlockBeingWrittenLength;
-  }
-
-  /** Read the block length from one of the datanodes. */
-  private long readBlockLength(LocatedBlock locatedblock) throws IOException {
-    assert locatedblock != null : "LocatedBlock cannot be null";
-    int replicaNotFoundCount = locatedblock.getLocations().length;
-    
-    final DfsClientConf conf = dfsClient.getConf();
-    for(DatanodeInfo datanode : locatedblock.getLocations()) {
-      ClientDatanodeProtocol cdp = null;
-      
-      try {
-        cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
-            dfsClient.getConfiguration(), conf.getSocketTimeout(),
-            conf.isConnectToDnViaHostname(), locatedblock);
-        
-        final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
-        
-        if (n >= 0) {
-          return n;
-        }
-      }
-      catch(IOException ioe) {
-        if (ioe instanceof RemoteException &&
-          (((RemoteException) ioe).unwrapRemoteException() instanceof
-            ReplicaNotFoundException)) {
-          // special case : replica might not be on the DN, treat as 0 length
-          replicaNotFoundCount--;
-        }
-        
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
-              + datanode + " for block " + locatedblock.getBlock(), ioe);
-        }
-      } finally {
-        if (cdp != null) {
-          RPC.stopProxy(cdp);
-        }
-      }
-    }
-
-    // Namenode told us about these locations, but none know about the replica
-    // means that we hit the race between pipeline creation start and end.
-    // we require all 3 because some other exception could have happened
-    // on a DN that has it.  we want to report that error
-    if (replicaNotFoundCount == 0) {
-      return 0;
-    }
-
-    throw new IOException("Cannot obtain block length for " + locatedblock);
-  }
-  
-  public long getFileLength() {
-    synchronized(infoLock) {
-      return locatedBlocks == null? 0:
-          locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
-    }
-  }
-
-  // Short circuit local reads are forbidden for files that are
-  // under construction.  See HDFS-2757.
-  boolean shortCircuitForbidden() {
-    synchronized(infoLock) {
-      return locatedBlocks.isUnderConstruction();
-    }
-  }
-
-  /**
-   * Returns the datanode from which the stream is currently reading.
-   */
-  public synchronized DatanodeInfo getCurrentDatanode() {
-    return currentNode;
-  }
-
-  /**
-   * Returns the block containing the target position. 
-   */
-  synchronized public ExtendedBlock getCurrentBlock() {
-    if (currentLocatedBlock == null){
-      return null;
-    }
-    return currentLocatedBlock.getBlock();
-  }
-
-  /**
-   * Return collection of blocks that has already been located.
-   */
-  public List<LocatedBlock> getAllBlocks() throws IOException {
-    return getBlockRange(0, getFileLength());
-  }
-
-  /**
-   * Get block at the specified position.
-   * Fetch it from the namenode if not cached.
-   * 
-   * @param offset block corresponding to this offset in file is returned
-   * @return located block
-   * @throws IOException
-   */
-  protected LocatedBlock getBlockAt(long offset) throws IOException {
-    synchronized(infoLock) {
-      assert (locatedBlocks != null) : "locatedBlocks is null";
-
-      final LocatedBlock blk;
-
-      //check offset
-      if (offset < 0 || offset >= getFileLength()) {
-        throw new IOException("offset < 0 || offset >= getFileLength(), offset="
-            + offset
-            + ", locatedBlocks=" + locatedBlocks);
-      }
-      else if (offset >= locatedBlocks.getFileLength()) {
-        // offset to the portion of the last block,
-        // which is not known to the name-node yet;
-        // getting the last block
-        blk = locatedBlocks.getLastLocatedBlock();
-      }
-      else {
-        // search cached blocks first
-        int targetBlockIdx = locatedBlocks.findBlock(offset);
-        if (targetBlockIdx < 0) { // block is not cached
-          targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
-          // fetch more blocks
-          final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
-          assert (newBlocks != null) : "Could not find target position " + offset;
-          locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
-        }
-        blk = locatedBlocks.get(targetBlockIdx);
-      }
-      return blk;
-    }
-  }
-
-  /** Fetch a block from namenode and cache it */
-  protected void fetchBlockAt(long offset) throws IOException {
-    synchronized(infoLock) {
-      int targetBlockIdx = locatedBlocks.findBlock(offset);
-      if (targetBlockIdx < 0) { // block is not cached
-        targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
-      }
-      // fetch blocks
-      final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
-      if (newBlocks == null) {
-        throw new IOException("Could not find target position " + offset);
-      }
-      locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
-    }
-  }
-
-  /**
-   * Get blocks in the specified range.
-   * Fetch them from the namenode if not cached. This function
-   * will not get a read request beyond the EOF.
-   * @param offset starting offset in file
-   * @param length length of data
-   * @return consequent segment of located blocks
-   * @throws IOException
-   */
-  private List<LocatedBlock> getBlockRange(long offset,
-      long length)  throws IOException {
-    // getFileLength(): returns total file length
-    // locatedBlocks.getFileLength(): returns length of completed blocks
-    if (offset >= getFileLength()) {
-      throw new IOException("Offset: " + offset +
-        " exceeds file length: " + getFileLength());
-    }
-    synchronized(infoLock) {
-      final List<LocatedBlock> blocks;
-      final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
-      final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
-      final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
-
-      if (readOffsetWithinCompleteBlk) {
-        //get the blocks of finalized (completed) block range
-        blocks = getFinalizedBlockRange(offset,
-          Math.min(length, lengthOfCompleteBlk - offset));
-      } else {
-        blocks = new ArrayList<LocatedBlock>(1);
-      }
-
-      // get the blocks from incomplete block range
-      if (readLengthPastCompleteBlk) {
-         blocks.add(locatedBlocks.getLastLocatedBlock());
-      }
-
-      return blocks;
-    }
-  }
-
-  /**
-   * Get blocks in the specified range.
-   * Includes only the complete blocks.
-   * Fetch them from the namenode if not cached.
-   */
-  private List<LocatedBlock> getFinalizedBlockRange(
-      long offset, long length) throws IOException {
-    synchronized(infoLock) {
-      assert (locatedBlocks != null) : "locatedBlocks is null";
-      List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
-      // search cached blocks first
-      int blockIdx = locatedBlocks.findBlock(offset);
-      if (blockIdx < 0) { // block is not cached
-        blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
-      }
-      long remaining = length;
-      long curOff = offset;
-      while(remaining > 0) {
-        LocatedBlock blk = null;
-        if(blockIdx < locatedBlocks.locatedBlockCount())
-          blk = locatedBlocks.get(blockIdx);
-        if (blk == null || curOff < blk.getStartOffset()) {
-          LocatedBlocks newBlocks;
-          newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
-          locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
-          continue;
-        }
-        assert curOff >= blk.getStartOffset() : "Block not found";
-        blockRange.add(blk);
-        long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
-        remaining -= bytesRead;
-        curOff += bytesRead;
-        blockIdx++;
-      }
-      return blockRange;
-    }
-  }
-
-  /**
-   * Open a DataInputStream to a DataNode so that it can be read from.
-   * We get block ID and the IDs of the destinations at startup, from the namenode.
-   */
-  private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
-    if (target >= getFileLength()) {
-      throw new IOException("Attempted to read past end of file");
-    }
-
-    // Will be getting a new BlockReader.
-    closeCurrentBlockReaders();
-
-    //
-    // Connect to best DataNode for desired Block, with potential offset
-    //
-    DatanodeInfo chosenNode = null;
-    int refetchToken = 1; // only need to get a new access token once
-    int refetchEncryptionKey = 1; // only need to get a new encryption key once
-    
-    boolean connectFailedOnce = false;
-
-    while (true) {
-      //
-      // Compute desired block
-      //
-      LocatedBlock targetBlock = getBlockAt(target);
-
-      // update current position
-      this.pos = target;
-      this.blockEnd = targetBlock.getStartOffset() +
-            targetBlock.getBlockSize() - 1;
-      this.currentLocatedBlock = targetBlock;
-
-      long offsetIntoBlock = target - targetBlock.getStartOffset();
-
-      DNAddrPair retval = chooseDataNode(targetBlock, null);
-      chosenNode = retval.info;
-      InetSocketAddress targetAddr = retval.addr;
-      StorageType storageType = retval.storageType;
-
-      try {
-        blockReader = getBlockReader(targetBlock, offsetIntoBlock,
-            targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
-            storageType, chosenNode);
-        if(connectFailedOnce) {
-          DFSClient.LOG.info("Successfully connected to " + targetAddr +
-                             " for " + targetBlock.getBlock());
-        }
-        return chosenNode;
-      } catch (IOException ex) {
-        if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
-          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
-              + "encryption key was invalid when connecting to " + targetAddr
-              + " : " + ex);
-          // The encryption key used is invalid.
-          refetchEncryptionKey--;
-          dfsClient.clearDataEncryptionKey();
-        } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
-          refetchToken--;
-          fetchBlockAt(target);
-        } else {
-          connectFailedOnce = true;
-          DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
-            + ", add to deadNodes and continue. " + ex, ex);
-          // Put chosen node into dead list, continue
-          addToDeadNodes(chosenNode);
-        }
-      }
-    }
-  }
-
-  protected BlockReader getBlockReader(LocatedBlock targetBlock,
-      long offsetInBlock, long length, InetSocketAddress targetAddr,
-      StorageType storageType, DatanodeInfo datanode) throws IOException {
-    ExtendedBlock blk = targetBlock.getBlock();
-    Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
-    CachingStrategy curCachingStrategy;
-    boolean shortCircuitForbidden;
-    synchronized (infoLock) {
-      curCachingStrategy = cachingStrategy;
-      shortCircuitForbidden = shortCircuitForbidden();
-    }
-    return new BlockReaderFactory(dfsClient.getConf()).
-        setInetSocketAddress(targetAddr).
-        setRemotePeerFactory(dfsClient).
-        setDatanodeInfo(datanode).
-        setStorageType(storageType).
-        setFileName(src).
-        setBlock(blk).
-        setBlockToken(accessToken).
-        setStartOffset(offsetInBlock).
-        setVerifyChecksum(verifyChecksum).
-        setClientName(dfsClient.clientName).
-        setLength(length).
-        setCachingStrategy(curCachingStrategy).
-        setAllowShortCircuitLocalReads(!shortCircuitForbidden).
-        setClientCacheContext(dfsClient.getClientContext()).
-        setUserGroupInformation(dfsClient.ugi).
-        setConfiguration(dfsClient.getConfiguration()).
-        build();
-  }
-
-  /**
-   * Close it down!
-   */
-  @Override
-  public synchronized void close() throws IOException {
-    if (!closed.compareAndSet(false, true)) {
-      DFSClient.LOG.debug("DFSInputStream has been closed already");
-      return;
-    }
-    dfsClient.checkOpen();
-
-    if ((extendedReadBuffers != null) && (!extendedReadBuffers.isEmpty())) {
-      final StringBuilder builder = new StringBuilder();
-      extendedReadBuffers.visitAll(new IdentityHashStore.Visitor<ByteBuffer, Object>() {
-        private String prefix = "";
-        @Override
-        public void accept(ByteBuffer k, Object v) {
-          builder.append(prefix).append(k);
-          prefix = ", ";
-        }
-      });
-      DFSClient.LOG.warn("closing file " + src + ", but there are still " +
-          "unreleased ByteBuffers allocated by read().  " +
-          "Please release " + builder.toString() + ".");
-    }
-    closeCurrentBlockReaders();
-    super.close();
-  }
-
-  @Override
-  public synchronized int read() throws IOException {
-    if (oneByteBuf == null) {
-      oneByteBuf = new byte[1];
-    }
-    int ret = read( oneByteBuf, 0, 1 );
-    return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
-  }
-
-  /**
-   * Wraps different possible read implementations so that readBuffer can be
-   * strategy-agnostic.
-   */
-  interface ReaderStrategy {
-    public int doRead(BlockReader blockReader, int off, int len)
-        throws ChecksumException, IOException;
-
-    /**
-     * Copy data from the src ByteBuffer into the read buffer.
-     * @param src The src buffer where the data is copied from
-     * @param offset Useful only when the ReadStrategy is based on a byte array.
-     *               Indicate the offset of the byte array for copy.
-     * @param length Useful only when the ReadStrategy is based on a byte array.
-     *               Indicate the length of the data to copy.
-     */
-    public int copyFrom(ByteBuffer src, int offset, int length);
-  }
-
-  protected void updateReadStatistics(ReadStatistics readStatistics,
-        int nRead, BlockReader blockReader) {
-    if (nRead <= 0) return;
-    synchronized(infoLock) {
-      if (blockReader.isShortCircuit()) {
-        readStatistics.addShortCircuitBytes(nRead);
-      } else if (blockReader.isLocal()) {
-        readStatistics.addLocalBytes(nRead);
-      } else {
-        readStatistics.addRemoteBytes(nRead);
-      }
-    }
-  }
-  
-  /**
-   * Used to read bytes into a byte[]
-   */
-  private class ByteArrayStrategy implements ReaderStrategy {
-    final byte[] buf;
-
-    public ByteArrayStrategy(byte[] buf) {
-      this.buf = buf;
-    }
-
-    @Override
-    public int doRead(BlockReader blockReader, int off, int len)
-          throws ChecksumException, IOException {
-      int nRead = blockReader.read(buf, off, len);
-      updateReadStatistics(readStatistics, nRead, blockReader);
-      return nRead;
-    }
-
-    @Override
-    public int copyFrom(ByteBuffer src, int offset, int length) {
-      ByteBuffer writeSlice = src.duplicate();
-      writeSlice.get(buf, offset, length);
-      return length;
-    }
-  }
-
-  /**
-   * Used to read bytes into a user-supplied ByteBuffer
-   */
-  protected class ByteBufferStrategy implements ReaderStrategy {
-    final ByteBuffer buf;
-    ByteBufferStrategy(ByteBuffer buf) {
-      this.buf = buf;
-    }
-
-    @Override
-    public int doRead(BlockReader blockReader, int off, int len)
-        throws ChecksumException, IOException {
-      int oldpos = buf.position();
-      int oldlimit = buf.limit();
-      boolean success = false;
-      try {
-        int ret = blockReader.read(buf);
-        success = true;
-        updateReadStatistics(readStatistics, ret, blockReader);
-        if (ret == 0) {
-          DFSClient.LOG.warn("zero");
-        }
-        return ret;
-      } finally {
-        if (!success) {
-          // Reset to original state so that retries work correctly.
-          buf.position(oldpos);
-          buf.limit(oldlimit);
-        }
-      } 
-    }
-
-    @Override
-    public int copyFrom(ByteBuffer src, int offset, int length) {
-      ByteBuffer writeSlice = src.duplicate();
-      int remaining = Math.min(buf.remaining(), writeSlice.remaining());
-      writeSlice.limit(writeSlice.position() + remaining);
-      buf.put(writeSlice);
-      return remaining;
-    }
-  }
-
-  /* This is a used by regular read() and handles ChecksumExceptions.
-   * name readBuffer() is chosen to imply similarity to readBuffer() in
-   * ChecksumFileSystem
-   */ 
-  private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
-      throws IOException {
-    IOException ioe;
-    
-    /* we retry current node only once. So this is set to true only here.
-     * Intention is to handle one common case of an error that is not a
-     * failure on datanode or client : when DataNode closes the connection
-     * since client is idle. If there are other cases of "non-errors" then
-     * then a datanode might be retried by setting this to true again.
-     */
-    boolean retryCurrentNode = true;
-
-    while (true) {
-      // retry as many times as seekToNewSource allows.
-      try {
-        return reader.doRead(blockReader, off, len);
-      } catch ( ChecksumException ce ) {
-        DFSClient.LOG.warn("Found Checksum error for "
-            + getCurrentBlock() + " from " + currentNode
-            + " at " + ce.getPos());        
-        ioe = ce;
-        retryCurrentNode = false;
-        // we want to remember which block replicas we have tried
-        addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
-            corruptedBlockMap);
-      } catch ( IOException e ) {
-        if (!retryCurrentNode) {
-          DFSClient.LOG.warn("Exception while reading from "
-              + getCurrentBlock() + " of " + src + " from "
-              + currentNode, e);
-        }
-        ioe = e;
-      }
-      boolean sourceFound = false;
-      if (retryCurrentNode) {
-        /* possibly retry the same node so that transient errors don't
-         * result in application level failures (e.g. Datanode could have
-         * closed the connection because the client is idle for too long).
-         */ 
-        sourceFound = seekToBlockSource(pos);
-      } else {
-        addToDeadNodes(currentNode);
-        sourceFound = seekToNewSource(pos);
-      }
-      if (!sourceFound) {
-        throw ioe;
-      }
-      retryCurrentNode = false;
-    }
-  }
-
-  protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
-    dfsClient.checkOpen();
-    if (closed.get()) {
-      throw new IOException("Stream closed");
-    }
-    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
-      = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
-    failures = 0;
-    if (pos < getFileLength()) {
-      int retries = 2;
-      while (retries > 0) {
-        try {
-          // currentNode can be left as null if previous read had a checksum
-          // error on the same block. See HDFS-3067
-          if (pos > blockEnd || currentNode == null) {
-            currentNode = blockSeekTo(pos);
-          }
-          int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
-          synchronized(infoLock) {
-            if (locatedBlocks.isLastBlockComplete()) {
-              realLen = (int) Math.min(realLen,
-                  locatedBlocks.getFileLength() - pos);
-            }
-          }
-          int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
-          
-          if (result >= 0) {
-            pos += result;
-          } else {
-            // got a EOS from reader though we expect more data on it.
-            throw new IOException("Unexpected EOS from the reader");
-          }
-          if (dfsClient.stats != null) {
-            dfsClient.stats.incrementBytesRead(result);
-          }
-          return result;
-        } catch (ChecksumException ce) {
-          throw ce;            
-        } catch (IOException e) {
-          if (retries == 1) {
-            DFSClient.LOG.warn("DFS Read", e);
-          }
-          blockEnd = -1;
-          if (currentNode != null) { addToDeadNodes(currentNode); }
-          if (--retries == 0) {
-            throw e;
-          }
-        } finally {
-          // Check if need to report block replicas corruption either read
-          // was successful or ChecksumException occured.
-          reportCheckSumFailure(corruptedBlockMap, 
-              currentLocatedBlock.getLocations().length);
-        }
-      }
-    }
-    return -1;
-  }
-
-  /**
-   * Read the entire buffer.
-   */
-  @Override
-  public synchronized int read(final byte buf[], int off, int len) throws IOException {
-    ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
-    TraceScope scope =
-        dfsClient.getPathTraceScope("DFSInputStream#byteArrayRead", src);
-    try {
-      return readWithStrategy(byteArrayReader, off, len);
-    } finally {
-      scope.close();
-    }
-  }
-
-  @Override
-  public synchronized int read(final ByteBuffer buf) throws IOException {
-    ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
-    TraceScope scope =
-        dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
-    try {
-      return readWithStrategy(byteBufferReader, 0, buf.remaining());
-    } finally {
-      scope.close();
-    }
-  }
-
-
-  /**
-   * Add corrupted block replica into map.
-   */
-  protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
-    Set<DatanodeInfo> dnSet = null;
-    if((corruptedBlockMap.containsKey(blk))) {
-      dnSet = corruptedBlockMap.get(blk);
-    }else {
-      dnSet = new HashSet<DatanodeInfo>();
-    }
-    if (!dnSet.contains(node)) {
-      dnSet.add(node);
-      corruptedBlockMap.put(blk, dnSet);
-    }
-  }
-
-  private DNAddrPair chooseDataNode(LocatedBlock block,
-      Collection<DatanodeInfo> ignoredNodes) throws IOException {
-    while (true) {
-      DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
-      if (result != null) {
-        return result;
-      } else {
-        String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
-          deadNodes, ignoredNodes);
-        String blockInfo = block.getBlock() + " file=" + src;
-        if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
-          String description = "Could not obtain block: " + blockInfo;
-          DFSClient.LOG.warn(description + errMsg
-              + ". Throwing a BlockMissingException");
-          throw new BlockMissingException(src, description,
-              block.getStartOffset());
-        }
-
-        DatanodeInfo[] nodes = block.getLocations();
-        if (nodes == null || nodes.length == 0) {
-          DFSClient.LOG.info("No node available for " + blockInfo);
-        }
-        DFSClient.LOG.info("Could not obtain " + block.getBlock()
-            + " from any node: " + errMsg
-            + ". Will get new block locations from namenode and retry...");
-        try {
-          // Introducing a random factor to the wait time before another retry.
-          // The wait time is dependent on # of failures and a random factor.
-          // At the first time of getting a BlockMissingException, the wait time
-          // is a random number between 0..3000 ms. If the first retry
-          // still fails, we will wait 3000 ms grace period before the 2nd retry.
-          // Also at the second retry, the waiting window is expanded to 6000 ms
-          // alleviating the request rate from the server. Similarly the 3rd retry
-          // will wait 6000ms grace period before retry and the waiting window is
-          // expanded to 9000ms. 
-          final int timeWindow = dfsClient.getConf().getTimeWindow();
-          double waitTime = timeWindow * failures +       // grace period for the last round of attempt
-              // expanding time window for each failure
-              timeWindow * (failures + 1) *
-              ThreadLocalRandom.current().nextDouble();
-          DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
-          Thread.sleep((long)waitTime);
-        } catch (InterruptedException iex) {
-        }
-        deadNodes.clear(); //2nd option is to remove only nodes[blockId]
-        openInfo(true);
-        block = refreshLocatedBlock(block);
-        failures++;
-      }
-    }
-  }
-
-  /**
-   * Get the best node from which to stream the data.
-   * @param block LocatedBlock, containing nodes in priority order.
-   * @param ignoredNodes Do not choose nodes in this array (may be null)
-   * @return The DNAddrPair of the best node. Null if no node can be chosen.
-   */
-  protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
-      Collection<DatanodeInfo> ignoredNodes) {
-    DatanodeInfo[] nodes = block.getLocations();
-    StorageType[] storageTypes = block.getStorageTypes();
-    DatanodeInfo chosenNode = null;
-    StorageType storageType = null;
-    if (nodes != null) {
-      for (int i = 0; i < nodes.length; i++) {
-        if (!deadNodes.containsKey(nodes[i])
-            && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
-          chosenNode = nodes[i];
-          // Storage types are ordered to correspond with nodes, so use the same
-          // index to get storage type.
-          if (storageTypes != null && i < storageTypes.length) {
-            storageType = storageTypes[i];
-          }
-          break;
-        }
-      }
-    }
-    if (chosenNode == null) {
-      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
-          " after checking nodes = " + Arrays.toString(nodes) +
-          ", ignoredNodes = " + ignoredNodes);
-      return null;
-    }
-    final String dnAddr =
-        chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
-    }
-    InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
-    return new DNAddrPair(chosenNode, targetAddr, storageType);
-  }
-
-  private static String getBestNodeDNAddrPairErrorString(
-      DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
-      DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
-    StringBuilder errMsgr = new StringBuilder(
-        " No live nodes contain current block ");
-    errMsgr.append("Block locations:");
-    for (DatanodeInfo datanode : nodes) {
-      errMsgr.append(" ");
-      errMsgr.append(datanode.toString());
-    }
-    errMsgr.append(" Dead nodes: ");
-    for (DatanodeInfo datanode : deadNodes.keySet()) {
-      errMsgr.append(" ");
-      errMsgr.append(datanode.toString());
-    }
-    if (ignoredNodes != null) {
-      errMsgr.append(" Ignored nodes: ");
-      for (DatanodeInfo datanode : ignoredNodes) {
-        errMsgr.append(" ");
-        errMsgr.append(datanode.toString());
-      }
-    }
-    return errMsgr.toString();
-  }
-
-  protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
-      byte[] buf, int offset,
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
-      throws IOException {
-    block = refreshLocatedBlock(block);
-    while (true) {
-      DNAddrPair addressPair = chooseDataNode(block, null);
-      try {
-        actualGetFromOneDataNode(addressPair, block, start, end,
-            buf, offset, corruptedBlockMap);
-        return;
-      } catch (IOException e) {
-        // Ignore. Already processed inside the function.
-        // Loop through to try the next node.
-      }
-    }
-  }
-
-  private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
-      final LocatedBlock block, final long start, final long end,
-      final ByteBuffer bb,
-      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
-      final int hedgedReadId) {
-    final Span parentSpan = Trace.currentSpan();
-    return new Callable<ByteBuffer>() {
-      @Override
-      public ByteBuffer call() throws Exception {
-        byte[] buf = bb.array();
-        int offset = bb.position();
-        TraceScope scope =
-            Trace.startSpan("hedgedRead" + hedgedReadId, parentSpan);
-        try {
-          actualGetFromOneDataNode(datanode, block, start, end, buf,
-              offset, corruptedBlockMap);
-          return bb;
-        } finally {
-          scope.close();
-        }
-      }
-    };
-  }
-
-  /**
-   * Used when reading contiguous blocks
-   */
-  private void actualGetFromOneDataNode(final DNAddrPair datanode,
-      LocatedBlock block, final long start, final long end, byte[] buf,
-      int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
-      throws IOException {
-    final int length = (int) (end - start + 1);
-    actualGetFromOneDataNode(datanode, block, start, end, buf,
-        new int[]{offset}, new int[]{length}, corruptedBlockMap);
-  }
-
-  /**
-   * Read data from one DataNode.
-   * @param datanode the datanode from which to read data
-   * @param block the located block containing the requested data
-   * @param startInBlk the startInBlk offset of the block
-   * @param endInBlk the endInBlk offset of the block
-   * @param buf the given byte array into which the data is read
-   * @param offsets the data may be read into multiple segments of the buf
-   *                (when reading a striped block). this array indicates the
-   *                offset of each buf segment.
-   * @param lengths the length of each buf segment
-   * @param corruptedBlockMap map recording list of datanodes with corrupted
-   *                          block replica
-   */
-  void actualGetFromOneDataNode(final DNAddrPair datanode,
-      LocatedBlock block, final long startInBlk, final long endInBlk,
-      byte[] buf, int[] offsets, int[] lengths,
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
-      throws IOException {
-    DFSClientFaultInjector.get().startFetchFromDatanode();
-    int refetchToken = 1; // only need to get a new access token once
-    int refetchEncryptionKey = 1; // only need to get a new encryption key once
-    final int len = (int) (endInBlk - startInBlk + 1);
-    checkReadPortions(offsets, lengths, len);
-
-    while (true) {
-      // cached block locations may have been updated by chooseDataNode()
-      // or fetchBlockAt(). Always get the latest list of locations at the
-      // start of the loop.
-      block = refreshLocatedBlock(block);
-      BlockReader reader = null;
-      try {
-        DFSClientFaultInjector.get().fetchFromDatanodeException();
-        reader = getBlockReader(block, startInBlk, len, datanode.addr,
-            datanode.storageType, datanode.info);
-        for (int i = 0; i < offsets.length; i++) {
-          int nread = reader.readAll(buf, offsets[i], lengths[i]);
-          updateReadStatistics(readStatistics, nread, reader);
-          if (nread != lengths[i]) {
-            throw new IOException("truncated return from reader.read(): " +
-                "excpected " + lengths[i] + ", got " + nread);
-          }
-        }
-        DFSClientFaultInjector.get().readFromDatanodeDelay();
-        return;
-      } catch (ChecksumException e) {
-        String msg = "fetchBlockByteRange(). Got a checksum exception for "
-            + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
-            + datanode.info;
-        DFSClient.LOG.warn(msg);
-        // we want to remember what we have tried
-        addIntoCorruptedBlockMap(block.getBlock(), datanode.info,
-            corruptedBlockMap);
-        addToDeadNodes(datanode.info);
-        throw new IOException(msg);
-      } catch (IOException e) {
-        if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
-          DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
-              + "encryption key was invalid when connecting to " + datanode.addr
-              + " : " + e);
-          // The encryption key used is invalid.
-          refetchEncryptionKey--;
-          dfsClient.clearDataEncryptionKey();
-        } else if (refetchToken > 0 && tokenRefetchNeeded(e, datanode.addr)) {
-          refetchToken--;
-          try {
-            fetchBlockAt(block.getStartOffset());
-          } catch (IOException fbae) {
-            // ignore IOE, since we can retry it later in a loop
-          }
-        } else {
-          String msg = "Failed to connect to " + datanode.addr + " for file "
-              + src + " for block " + block.getBlock() + ":" + e;
-          DFSClient.LOG.warn("Connection failure: " + msg, e);
-          addToDeadNodes(datanode.info);
-          throw new IOException(msg);
-        }
-      } finally {
-        if (reader != null) {
-          reader.close();
-        }
-      }
-    }
-  }
-
-  /**
-   * Refresh cached block locations.
-   * @param block The currently cached block locations
-   * @return Refreshed block locations
-   * @throws IOException
-   */
-  protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
-      throws IOException {
-    return getBlockAt(block.getStartOffset());
-  }
-
-  /**
-   * This method verifies that the read portions are valid and do not overlap
-   * with each other.
-   */
-  private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
-    Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
-    int sum = 0;
-    for (int i = 0; i < lengths.length; i++) {
-      if (i > 0) {
-        int gap = offsets[i] - offsets[i - 1];
-        // make sure read portions do not overlap with each other
-        Preconditions.checkArgument(gap >= lengths[i - 1]);
-      }
-      sum += lengths[i];
-    }
-    Preconditions.checkArgument(sum == totalLen);
-  }
-
-  /**
-   * Like {@link #fetchBlockByteRange}except we start up a second, parallel,
-   * 'hedged' read if the first read is taking longer than configured amount of
-   * time. We then wait on which ever read returns first.
-   */
-  private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
-      long end, byte[] buf, int offset,
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
-      throws IOException {
-    final DfsClientConf conf = dfsClient.getConf();
-    ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
-    CompletionService<ByteBuffer> hedgedService =
-        new ExecutorCompletionService<ByteBuffer>(
-        dfsClient.getHedgedReadsThreadPool());
-    ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
-    ByteBuffer bb = null;
-    int len = (int) (end - start + 1);
-    int hedgedReadId = 0;
-    block = refreshLocatedBlock(block);
-    while (true) {
-      // see HDFS-6591, this metric is used to verify/catch unnecessary loops
-      hedgedReadOpsLoopNumForTesting++;
-      DNAddrPair chosenNode = null;
-      // there is no request already executing.
-      if (futures.isEmpty()) {
-        // chooseDataNode is a commitment. If no node, we go to
-        // the NN to reget block locations. Only go here on first read.
-        chosenNode = chooseDataNode(block, ignored);
-        bb = ByteBuffer.wrap(buf, offset, len);
-        Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
-            chosenNode, block, start, end, bb,
-            corruptedBlockMap, hedgedReadId++);
-        Future<ByteBuffer> firstRequest = hedgedService
-            .submit(getFromDataNodeCallable);
-        futures.add(firstRequest);
-        try {
-          Future<ByteBuffer> future = hedgedService.poll(
-              conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
-          if (future != null) {
-            future.get();
-            return;
-          }
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
-                + "ms to read from " + chosenNode.info
-                + "; spawning hedged read");
-          }
-          // Ignore this node on next go around.
-          ignored.add(chosenNode.info);
-          dfsClient.getHedgedReadMetrics().incHedgedReadOps();
-          continue; // no need to refresh block locations
-        } catch (InterruptedException e) {
-          // Ignore
-        } catch (ExecutionException e) {
-          // Ignore already logged in the call.
-        }
-      } else {
-        // We are starting up a 'hedged' read. We have a read already
-        // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
-        // If no nodes to do hedged reads against, pass.
-        try {
-          chosenNode = getBestNodeDNAddrPair(block, ignored);
-          if (chosenNode == null) {
-            chosenNode = chooseDataNode(block, ignored);
-          }
-          bb = ByteBuffer.allocate(len);
-          Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
-              chosenNode, block, start, end, bb,
-              corruptedBlockMap, hedgedReadId++);
-          Future<ByteBuffer> oneMoreRequest = hedgedService
-              .submit(getFromDataNodeCallable);
-          futures.add(oneMoreRequest);
-        } catch (IOException ioe) {
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Failed getting node for hedged read: "
-                + ioe.getMessage());
-          }
-        }
-        // if not succeeded. Submit callables for each datanode in a loop, wait
-        // for a fixed interval and get the result from the fastest one.
-        try {
-          ByteBuffer result = getFirstToComplete(hedgedService, futures);
-          // cancel the rest.
-          cancelAll(futures);
-          if (result.array() != buf) { // compare the array pointers
-            dfsClient.getHedgedReadMetrics().incHedgedReadWins();
-            System.arraycopy(result.array(), result.position(), buf, offset,
-                len);
-          } else {
-            dfsClient.getHedgedReadMetrics().incHedgedReadOps();
-          }
-          return;
-        } catch (InterruptedException ie) {
-          // Ignore and retry
-        }
-        // We got here if exception. Ignore this node on next go around IFF
-        // we found a chosenNode to hedge read against.
-        if (chosenNode != null && chosenNode.info != null) {
-          ignored.add(chosenNode.info);
-        }
-      }
-    }
-  }
-
-  @VisibleForTesting
-  public long getHedgedReadOpsLoopNumForTesting() {
-    return hedgedReadOpsLoopNumForTesting;
-  }
-
-  private ByteBuffer getFirstToComplete(
-      CompletionService<ByteBuffer> hedgedService,
-      ArrayList<Future<ByteBuffer>> futures) throws InterruptedException {
-    if (futures.isEmpty()) {
-      throw new InterruptedException("let's retry");
-    }
-    Future<ByteBuffer> future = null;
-    try {
-      future = hedgedService.take();
-      ByteBuffer bb = future.get();
-      futures.remove(future);
-      return bb;
-    } catch (ExecutionException e) {
-      // already logged in the Callable
-      futures.remove(future);
-    } catch (CancellationException ce) {
-      // already logged in the Callable
-      futures.remove(future);
-    }
-
-    throw new InterruptedException("let's retry");
-  }
-
-  private void cancelAll(List<Future<ByteBuffer>> futures) {
-    for (Future<ByteBuffer> future : futures) {
-      // Unfortunately, hdfs reads do not take kindly to interruption.
-      // Threads return a variety of interrupted-type exceptions but
-      // also complaints about invalid pbs -- likely because read
-      // is interrupted before gets whole pb.  Also verbose WARN
-      // logging.  So, for now, do not interrupt running read.
-      future.cancel(false);
-    }
-  }
-
-  /**
-   * Should the block access token be refetched on an exception
-   * 
-   * @param ex Exception received
-   * @param targetAddr Target datanode address from where exception was received
-   * @return true if block access token has expired or invalid and it should be
-   *         refetched
-   */
-  protected static boolean tokenRefetchNeeded(IOException ex,
-      InetSocketAddress targetAddr) {
-    /*
-     * Get a new access token and retry. Retry is needed in 2 cases. 1)
-     * When both NN and DN re-started while DFSClient holding a cached
-     * access token. 2) In the case that NN fails to update its
-     * access key at pre-set interval (by a wide margin) and
-     * subsequently restarts. In this case, DN re-registers itself with
-     * NN and receives a new access key, but DN will delete the old
-     * access key from its memory since it's considered expired based on
-     * the estimated expiration date.
-     */
-    if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
-      DFSClient.LOG.info("Access token was invalid when connecting to "
-          + targetAddr + " : " + ex);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Read bytes starting from the specified position.
-   * 
-   * @param position start read from this position
-   * @param buffer read buffer
-   * @param offset offset into buffer
-   * @param length number of bytes to read
-   * 
-   * @return actual number of bytes read
-   */
-  @Override
-  public int read(long position, byte[] buffer, int offset, int length)
-      throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("DFSInputStream#byteArrayPread", src);
-    try {
-      return pread(position, buffer, offset, length);
-    } finally {
-      scope.close();
-    }
-  }
-
-  private int pread(long position, byte[] buffer, int offset, int length)
-      throws IOException {
-    // sanity checks
-    dfsClient.checkOpen();
-    if (closed.get()) {
-      throw new IOException("Stream closed");
-    }
-    failures = 0;
-    long filelen = getFileLength();
-    if ((position < 0) || (position >= filelen)) {
-      return -1;
-    }
-    int realLen = length;
-    if ((position + length) > filelen) {
-      realLen = (int)(filelen - position);
-    }
-    
-    // determine the block and byte range within the block
-    // corresponding to position and realLen
-    List<LocatedBlock> blockRange = getBlockRange(position, realLen);
-    int remaining = realLen;
-    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
-      = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
-    for (LocatedBlock blk : blockRange) {
-      long targetStart = position - blk.getStartOffset();
-      long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
-      try {
-        if (dfsClient.isHedgedReadsEnabled()) {
-          hedgedFetchBlockByteRange(blk, targetStart,
-              targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
-        } else {
-          fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
-              buffer, offset, corruptedBlockMap);
-        }
-      } finally {
-        // Check and report if any block replicas are corrupted.
-        // BlockMissingException may be caught if all block replicas are
-        // corrupted.
-        reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
-      }
-
-      remaining -= bytesToRead;
-      position += bytesToRead;
-      offset += bytesToRead;
-    }
-    assert remaining == 0 : "Wrong number of bytes read.";
-    if (dfsClient.stats != null) {
-      dfsClient.stats.incrementBytesRead(realLen);
-    }
-    return realLen;
-  }
-  
-  /**
-   * DFSInputStream reports checksum failure.
-   * Case I : client has tried multiple data nodes and at least one of the
-   * attempts has succeeded. We report the other failures as corrupted block to
-   * namenode. 
-   * Case II: client has tried out all data nodes, but all failed. We
-   * only report if the total number of replica is 1. We do not
-   * report otherwise since this maybe due to the client is a handicapped client
-   * (who can not read).
-   * @param corruptedBlockMap map of corrupted blocks
-   * @param dataNodeCount number of data nodes who contains the block replicas
-   */
-  protected void reportCheckSumFailure(
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap, 
-      int dataNodeCount) {
-    if (corruptedBlockMap.isEmpty()) {
-      return;
-    }
-    Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
-        .entrySet().iterator();
-    Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
-    ExtendedBlock blk = entry.getKey();
-    Set<DatanodeInfo> dnSet = entry.getValue();
-    if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
-        || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
-      DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
-      int i = 0;
-      for (DatanodeInfo dn:dnSet) {
-        locs[i++] = dn;
-      }
-      LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
-      dfsClient.reportChecksumFailure(src, lblocks);
-    }
-    corruptedBlockMap.clear();
-  }
-
-  @Override
-  public long skip(long n) throws IOException {
-    if ( n > 0 ) {
-      long curPos = getPos();
-      long fileLen = getFileLength();
-      if( n+curPos > fileLen ) {
-        n = fileLen - curPos;
-      }
-      seek(curPos+n);
-      return n;
-    }
-    return n < 0 ? -1 : 0;
-  }
-
-  /**
-   * Seek to a new arbitrary location
-   */
-  @Override
-  public synchronized void seek(long targetPos) throws IOException {
-    if (targetPos > getFileLength()) {
-      throw new EOFException("Cannot seek after EOF");
-    }
-    if (targetPos < 0) {
-      throw new EOFException("Cannot seek to negative offset");
-    }
-    if (closed.get()) {
-      throw new IOException("Stream is closed!");
-    }
-    boolean done = false;
-    if (pos <= targetPos && targetPos <= blockEnd) {
-      //
-      // If this seek is to a positive position in the current
-      // block, and this piece of data might already be lying in
-      // the TCP buffer, then just eat up the intervening data.
-      //
-      int diff = (int)(targetPos - pos);
-      if (diff <= blockReader.available()) {
-        try {
-          pos += blockReader.skip(diff);
-          if (pos == targetPos) {
-            done = true;
-          } else {
-            // The range was already checked. If the block reader returns
-            // something unexpected instead of throwing an exception, it is
-            // most likely a bug. 
-            String errMsg = "BlockReader failed to seek to " + 
-                targetPos + ". Instead, it seeked to " + pos + ".";
-            DFSClient.LOG.warn(errMsg);
-            throw new IOException(errMsg);
-          }
-        } catch (IOException e) {//make following read to retry
-          if(DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Exception while seek to " + targetPos
-                + " from " + getCurrentBlock() + " of " + src + " from "
-                + currentNode, e);
-          }
-        }
-      }
-    }
-    if (!done) {
-      pos = targetPos;
-      blockEnd = -1;
-    }
-  }
-
-  /**
-   * Same as {@link #seekToNewSource(long)} except that it does not exclude
-   * the current datanode and might connect to the same node.
-   */
-  private boolean seekToBlockSource(long targetPos)
-                                                 throws IOException {
-    currentNode = blockSeekTo(targetPos);
-    return true;
-  }
-  
-  /**
-   * Seek to given position on a node other than the current node.  If
-   * a node other than the current node is found, then returns true. 
-   * If another node could not be found, then returns false.
-   */
-  @Override
-  public synchronized boolean seekToNewSource(long targetPos) throws IOException {
-    if (currentNode == null) {
-      return seekToBlockSource(targetPos);
-    }
-    boolean markedDead = deadNodes.containsKey(currentNode);
-    addToDeadNodes(currentNode);
-    DatanodeInfo oldNode = currentNode;
-    DatanodeInfo newNode = blockSeekTo(targetPos);
-    if (!markedDead) {
-      /* remove it from deadNodes. blockSeekTo could have cleared 
-       * deadNodes and added currentNode again. Thats ok. */
-      deadNodes.remove(oldNode);
-    }
-    if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
-      currentNode = newNode;
-      return true;
-    } else {
-      return false;
-    }
-  }
-      
-  /**
-   */
-  @Override
-  public synchronized long getPos() {
-    return pos;
-  }
-
-  /** Return the size of the remaining available bytes
-   * if the size is less than or equal to {@link Integer#MAX_VALUE},
-   * otherwise, return {@link Integer#MAX_VALUE}.
-   */
-  @Override
-  public synchronized int available() throws IOException {
-    if (closed.get()) {
-      throw new IOException("Stream closed");
-    }
-
-    final long remaining = getFileLength() - pos;
-    return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
-  }
-
-  /**
-   * We definitely don't support marks
-   */
-  @Override
-  public boolean markSupported() {
-    return false;
-  }
-  @Override
-  public void mark(int readLimit) {
-  }
-  @Override
-  public void reset() throws IOException {
-    throw new IOException("Mark/reset not supported");
-  }
-
-  /** Utility class to encapsulate data node info and its address. */
-  static final class DNAddrPair {
-    final DatanodeInfo info;
-    final InetSocketAddress addr;
-    final StorageType storageType;
-
-    DNAddrPair(DatanodeInfo info, InetSocketAddress addr,
-        StorageType storageType) {
-      this.info = info;
-      this.addr = addr;
-      this.storageType = storageType;
-    }
-  }
-
-  /**
-   * Get statistics about the reads which this DFSInputStream has done.
-   */
-  public ReadStatistics getReadStatistics() {
-    synchronized(infoLock) {
-      return new ReadStatistics(readStatistics);
-    }
-  }
-
-  /**
-   * Clear statistics about the reads which this DFSInputStream has done.
-   */
-  public void clearReadStatistics() {
-    synchronized(infoLock) {
-      readStatistics.clear();
-    }
-  }
-
-  public FileEncryptionInfo getFileEncryptionInfo() {
-    synchronized(infoLock) {
-      return fileEncryptionInfo;
-    }
-  }
-
-  protected void closeCurrentBlockReaders() {
-    if (blockReader == null) return;
-    // Close the current block reader so that the new caching settings can 
-    // take effect immediately.
-    try {
-      blockReader.close();
-    } catch (IOException e) {
-      DFSClient.LOG.error("error closing blockReader", e);
-    }
-    blockReader = null;
-    blockEnd = -1;
-  }
-
-  @Override
-  public synchronized void setReadahead(Long readahead)
-      throws IOException {
-    synchronized (infoLock) {
-      this.cachingStrategy =
-          new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
-    }
-    closeCurrentBlockReaders();
-  }
-
-  @Override
-  public synchronized void setDropBehind(Boolean dropBehind)
-      throws IOException {
-    synchronized (infoLock) {
-      this.cachingStrategy =
-          new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
-    }
-    closeCurrentBlockReaders();
-  }
-
-  /**
-   * The immutable empty buffer we return when we reach EOF when doing a
-   * zero-copy read.
-   */
-  private static final ByteBuffer EMPTY_BUFFER =
-    ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
-
-  @Override
-  public synchronized ByteBuffer read(ByteBufferPool bufferPool,
-      int maxLength, EnumSet<ReadOption> opts) 
-          throws IOException, UnsupportedOperationException {
-    if (maxLength == 0) {
-      return EMPTY_BUFFER;
-    } else if (maxLength < 0) {
-      throw new IllegalArgumentException("can't read a negative " +
-          "number of bytes.");
-    }
-    if ((blockReader == null) || (blockEnd == -1)) {
-      if (pos >= getFileLength()) {
-        return null;
-      }
-      /*
-       * If we don't have a blockReader, or the one we have has no more bytes
-       * left to read, we call seekToBlockSource to get a new blockReader and
-       * recalculate blockEnd.  Note that we assume we're not at EOF here
-       * (we check this above).
-       */
-      if ((!seekToBlockSource(pos)) || (blockReader == null)) {
-        throw new IOException("failed to allocate new BlockReader " +
-            "at position " + pos);
-      }
-    }
-    ByteBuffer buffer = null;
-    if (dfsClient.getConf().getShortCircuitConf().isShortCircuitMmapEnabled()) {
-      buffer = tryReadZeroCopy(maxLength, opts);
-    }
-    if (buffer != null) {
-      return buffer;
-    }
-    buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
-    if (buffer != null) {
-      getExtendedReadBuffers().put(buffer, bufferPool);
-    }
-    return buffer;
-  }
-
-  private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
-      EnumSet<ReadOption> opts) throws IOException {
-    // Copy 'pos' and 'blockEnd' to local variables to make it easier for the
-    // JVM to optimize this function.
-    final long curPos = pos;
-    final long curEnd = blockEnd;
-    final long blockStartInFile = currentLocatedBlock.getStartOffset();
-    final long blockPos = curPos - blockStartInFile;
-
-    // Shorten this read if the end of the block is nearby.
-    long length63;
-    if ((curPos + maxLength) <= (curEnd + 1)) {
-      length63 = maxLength;
-    } else {
-      length63 = 1 + curEnd - curPos;
-      if (length63 <= 0) {
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
-            curPos + " of " + src + "; " + length63 + " bytes left in block.  " +
-            "blockPos=" + blockPos + "; curPos=" + curPos +
-            "; curEnd=" + curEnd);
-        }
-        return null;
-      }
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Reducing read length from " + maxLength +
-            " to " + length63 + " to avoid going more than one byte " +
-            "past the end of the block.  blockPos=" + blockPos +
-            "; curPos=" + curPos + "; curEnd=" + curEnd);
-      }
-    }
-    // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
-    int length;
-    if (blockPos + length63 <= Integer.MAX_VALUE) {
-      length = (int)length63;
-    } else {
-      long length31 = Integer.MAX_VALUE - blockPos;
-      if (length31 <= 0) {
-        // Java ByteBuffers can't be longer than 2 GB, because they use
-        // 4-byte signed integers to represent capacity, etc.
-        // So we can't mmap the parts of the block higher than the 2 GB offset.
-        // FIXME: we could work around this with multiple memory maps.
-        // See HDFS-5101.
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
-            curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
-            "exceeded.  blockPos=" + blockPos + ", curEnd=" + curEnd);
-        }
-        return null;
-      }
-      length = (int)length31;
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Reducing read length from " + maxLength +
-            " to " + length + " to avoid 31-bit limit.  " +
-            "blockPos=" + blockPos + "; curPos=" + curPos +
-            "; curEnd=" + curEnd);
-      }
-    }
-    final ClientMmap clientMmap = blockReader.getClientMmap(opts);
-    if (clientMmap == null) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
-          curPos + " of " + src + "; BlockReader#getClientMmap returned " +
-          "null.");
-      }
-      return null;
-    }
-    boolean success = false;
-    ByteBuffer buffer;
-    try {
-      seek(curPos + length);
-      buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
-      buffer.position((int)blockPos);
-      buffer.limit((int)(blockPos + length));
-      getExtendedReadBuffers().put(buffer, clientMmap);
-      synchronized (infoLock) {
-        readStatistics.addZeroCopyBytes(length);
-      }
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("readZeroCopy read " + length + 
-            " bytes from offset " + curPos + " via the zero-copy read " +
-            "path.  blockEnd = " + blockEnd);
-      }
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeQuietly(clientMmap);
-      }
-    }
-    return buffer;
-  }
-
-  @Override
-  public synchronized void releaseBuffer(ByteBuffer buffer) {
-    if (buffer == EMPTY_BUFFER) return;
-    Object val = getExtendedReadBuffers().remove(buffer);
-    if (val == null) {
-      throw new IllegalArgumentException("tried to release a buffer " +
-          "that was not created by this stream, " + buffer);
-    }
-    if (val instanceof ClientMmap) {
-      IOUtils.closeQuietly((ClientMmap)val);
-    } else if (val instanceof ByteBufferPool) {
-      ((ByteBufferPool)val).putBuffer(buffer);
-    }
-  }
-
-  @Override
-  public synchronized void unbuffer() {
-    closeCurrentBlockReaders();
-  }
-}


[12/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by wh...@apache.org.
HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/94cbb6d1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/94cbb6d1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/94cbb6d1

Branch: refs/heads/branch-2
Commit: 94cbb6d16483ba011b7104565b4084bf2a3eb6e6
Parents: b46e4ce
Author: Haohui Mai <wh...@apache.org>
Authored: Sat Sep 26 11:08:25 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Sat Sep 26 11:16:50 2015 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   24 +
 .../apache/hadoop/fs/BlockStorageLocation.java  |   52 +
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 +
 .../java/org/apache/hadoop/fs/HdfsVolumeId.java |   73 +
 .../java/org/apache/hadoop/fs/VolumeId.java     |   40 +
 .../hadoop/hdfs/BlockMissingException.java      |   65 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  893 +++++
 .../hadoop/hdfs/BlockStorageLocationUtil.java   |  369 ++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3203 ++++++++++++++++++
 .../hadoop/hdfs/DFSClientFaultInjector.java     |   60 +
 .../hadoop/hdfs/DFSHedgedReadMetrics.java       |   58 +
 .../hadoop/hdfs/DFSInotifyEventInputStream.java |  239 ++
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 1915 +++++++++++
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  923 +++++
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  345 ++
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   24 +
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 1904 +++++++++++
 .../hadoop/hdfs/HdfsConfigurationLoader.java    |   44 +
 .../apache/hadoop/hdfs/RemotePeerFactory.java   |   43 +
 .../hdfs/UnknownCipherSuiteException.java       |   35 +
 .../UnknownCryptoProtocolVersionException.java  |   38 +
 .../org/apache/hadoop/hdfs/XAttrHelper.java     |  174 +
 .../hadoop/hdfs/client/HdfsDataInputStream.java |  113 +
 .../hdfs/client/HdfsDataOutputStream.java       |  112 +
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  524 +++
 .../hdfs/inotify/MissingEventsException.java    |   54 +
 .../hadoop/hdfs/protocol/AclException.java      |   39 +
 .../hdfs/protocol/CacheDirectiveIterator.java   |  130 +
 .../hadoop/hdfs/protocol/CachePoolIterator.java |   63 +
 .../hdfs/protocol/EncryptionZoneIterator.java   |   64 +
 .../QuotaByStorageTypeExceededException.java    |   56 +
 .../hdfs/protocol/UnresolvedPathException.java  |   87 +
 .../datatransfer/ReplaceDatanodeOnFailure.java  |  200 ++
 .../datanode/ReplicaNotFoundException.java      |   53 +
 .../namenode/RetryStartFileException.java       |   36 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |    3 +
 .../dev-support/findbugsExcludeFile.xml         |   19 -
 .../apache/hadoop/fs/BlockStorageLocation.java  |   52 -
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 -
 .../java/org/apache/hadoop/fs/HdfsVolumeId.java |   73 -
 .../java/org/apache/hadoop/fs/VolumeId.java     |   40 -
 .../hadoop/hdfs/BlockMissingException.java      |   65 -
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  892 -----
 .../hadoop/hdfs/BlockStorageLocationUtil.java   |  369 --
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3200 -----------------
 .../hadoop/hdfs/DFSClientFaultInjector.java     |   57 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    2 -
 .../hadoop/hdfs/DFSHedgedReadMetrics.java       |   58 -
 .../hadoop/hdfs/DFSInotifyEventInputStream.java |  239 --
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 1915 -----------
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  923 -----
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  345 --
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   23 -
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 1903 -----------
 .../apache/hadoop/hdfs/HdfsConfiguration.java   |   11 +-
 .../apache/hadoop/hdfs/RemotePeerFactory.java   |   43 -
 .../hdfs/UnknownCipherSuiteException.java       |   35 -
 .../UnknownCryptoProtocolVersionException.java  |   38 -
 .../org/apache/hadoop/hdfs/XAttrHelper.java     |  174 -
 .../hadoop/hdfs/client/HdfsDataInputStream.java |  113 -
 .../hdfs/client/HdfsDataOutputStream.java       |  112 -
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  524 ---
 .../hdfs/inotify/MissingEventsException.java    |   54 -
 .../hadoop/hdfs/protocol/AclException.java      |   39 -
 .../hdfs/protocol/CacheDirectiveIterator.java   |  130 -
 .../hadoop/hdfs/protocol/CachePoolIterator.java |   63 -
 .../hdfs/protocol/EncryptionZoneIterator.java   |   64 -
 .../QuotaByStorageTypeExceededException.java    |   56 -
 .../hdfs/protocol/UnresolvedPathException.java  |   87 -
 .../datatransfer/ReplaceDatanodeOnFailure.java  |  200 --
 .../hadoop/hdfs/server/balancer/Dispatcher.java |    3 +-
 .../hdfs/server/datanode/BlockReceiver.java     |    4 +-
 .../hdfs/server/datanode/BlockSender.java       |    4 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |    3 +-
 .../hdfs/server/datanode/DataXceiver.java       |    6 +-
 .../datanode/ReplicaNotFoundException.java      |   53 -
 .../datanode/fsdataset/impl/BlockPoolSlice.java |    4 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |    8 +-
 .../impl/RamDiskAsyncLazyPersistService.java    |    4 +-
 .../namenode/RetryStartFileException.java       |   36 -
 .../hdfs/server/namenode/TransferFsImage.java   |    4 +-
 .../datanode/TestFiDataTransferProtocol2.java   |    1 -
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |    2 +-
 .../hdfs/MiniDFSClusterWithNodeGroup.java       |    2 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |    1 -
 .../TestClientProtocolForPipelineRecovery.java  |    6 +-
 .../apache/hadoop/hdfs/TestCrcCorruption.java   |    2 +-
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |    8 +-
 .../org/apache/hadoop/hdfs/TestFileStatus.java  |    2 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |   10 +-
 90 files changed, 12143 insertions(+), 12087 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
index 515da24..41a8564 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@ -32,4 +32,28 @@
     <Method name="allocSlot" />
     <Bug pattern="UL_UNRELEASED_LOCK" />
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.DFSInputStream"/>
+    <Field name="tcpReadsDisabledForTesting"/>
+    <Bug pattern="MS_SHOULD_BE_FINAL"/>
+  </Match>
+
+  <!--
+    ResponseProccessor is thread that is designed to catch RuntimeException.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.DataStreamer$ResponseProcessor" />
+    <Method name="run" />
+    <Bug pattern="REC_CATCH_EXCEPTION" />
+  </Match>
+
+  <!--
+    We use a separate lock to guard cachingStrategy in order to separate
+    locks for p-reads from seek + read invocations.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.DFSInputStream" />
+    <Field name="cachingStrategy" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
new file mode 100644
index 0000000..2200994
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Wrapper for {@link BlockLocation} that also adds {@link VolumeId} volume
+ * location information for each replica.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Public
+@Deprecated
+public class BlockStorageLocation extends BlockLocation {
+
+  private final VolumeId[] volumeIds;
+
+  public BlockStorageLocation(BlockLocation loc, VolumeId[] volumeIds)
+      throws IOException {
+    // Initialize with data from passed in BlockLocation
+    super(loc.getNames(), loc.getHosts(), loc.getTopologyPaths(), loc
+        .getOffset(), loc.getLength(), loc.isCorrupt());
+    this.volumeIds = volumeIds;
+  }
+
+  /**
+   * Gets the list of {@link VolumeId} corresponding to the block's replicas.
+   * 
+   * @return volumeIds list of VolumeId for the block's replicas
+   */
+  public VolumeId[] getVolumeIds() {
+    return volumeIds;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
new file mode 100644
index 0000000..0ccacda
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+
+/**
+ * Wrapper for {@link BlockLocation} that also includes a {@link LocatedBlock},
+ * allowing more detailed queries to the datanode about a block.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HdfsBlockLocation extends BlockLocation {
+
+  private final LocatedBlock block;
+  
+  public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) 
+      throws IOException {
+    // Initialize with data from passed in BlockLocation
+    super(loc);
+    this.block = block;
+  }
+  
+  public LocatedBlock getLocatedBlock() {
+    return block;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
new file mode 100644
index 0000000..6e9d3d7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * HDFS-specific volume identifier which implements {@link VolumeId}. Can be
+ * used to differentiate between the data directories on a single datanode. This
+ * identifier is only unique on a per-datanode basis.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Public
+public class HdfsVolumeId implements VolumeId {
+  
+  private final byte[] id;
+
+  public HdfsVolumeId(byte[] id) {
+    Preconditions.checkNotNull(id, "id cannot be null");
+    this.id = id;
+  }
+
+  @Override
+  public int compareTo(VolumeId arg0) {
+    if (arg0 == null) {
+      return 1;
+    }
+    return hashCode() - arg0.hashCode();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(id).toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null || obj.getClass() != getClass()) {
+      return false;
+    }
+    if (obj == this) {
+      return true;
+    }
+    HdfsVolumeId that = (HdfsVolumeId) obj;
+    return new EqualsBuilder().append(this.id, that.id).isEquals();
+  }
+
+  @Override
+  public String toString() {
+    return StringUtils.byteToHexString(id);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java
new file mode 100644
index 0000000..e56e304
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Opaque interface that identifies a disk location. Subclasses
+ * should implement {@link Comparable} and override both equals and hashCode.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Public
+public interface VolumeId extends Comparable<VolumeId> {
+
+  @Override
+  abstract public int compareTo(VolumeId arg0);
+
+  @Override
+  abstract public int hashCode();
+
+  @Override
+  abstract public boolean equals(Object obj);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
new file mode 100644
index 0000000..7bba8a4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/** 
+  * This exception is thrown when a read encounters a block that has no locations
+  * associated with it.
+  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockMissingException extends IOException {
+
+  private static final long serialVersionUID = 1L;
+
+  private final String filename;
+  private final long   offset;
+
+  /**
+   * An exception that indicates that file was corrupted.
+   * @param filename name of corrupted file
+   * @param description a description of the corruption details
+   */
+  public BlockMissingException(String filename, String description, long offset) {
+    super(description);
+    this.filename = filename;
+    this.offset = offset;
+  }
+
+  /**
+   * Returns the name of the corrupted file.
+   * @return name of corrupted file
+   */
+  public String getFile() {
+    return filename;
+  }
+
+  /**
+   * Returns the offset at which this file is corrupted
+   * @return offset of corrupted file
+   */
+  public long getOffset() {
+    return offset;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
new file mode 100644
index 0000000..69e9da2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -0,0 +1,893 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.InetSocketAddress;
+import java.util.List;
+
+import com.google.common.io.ByteArrayDataOutput;
+import com.google.common.io.ByteStreams;
+import org.apache.commons.lang.mutable.MutableBoolean;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
+import org.apache.hadoop.hdfs.net.DomainPeer;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplicaInfo;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.PerformanceAdvisory;
+import org.apache.hadoop.util.Time;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** 
+ * Utility class to create BlockReader implementations.
+ */
+@InterfaceAudience.Private
+public class BlockReaderFactory implements ShortCircuitReplicaCreator {
+  static final Logger LOG = LoggerFactory.getLogger(BlockReaderFactory.class);
+
+  public static class FailureInjector {
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      // do nothing
+    }
+    public boolean getSupportsReceiptVerification() {
+      return true;
+    }
+  }
+
+  @VisibleForTesting
+  static ShortCircuitReplicaCreator
+      createShortCircuitReplicaInfoCallback = null;
+
+  private final DfsClientConf conf;
+
+  /**
+   * Injects failures into specific operations during unit tests.
+   */
+  private static FailureInjector failureInjector = new FailureInjector();
+
+  /**
+   * The file name, for logging and debugging purposes.
+   */
+  private String fileName;
+
+  /**
+   * The block ID and block pool ID to use.
+   */
+  private ExtendedBlock block;
+
+  /**
+   * The block token to use for security purposes.
+   */
+  private Token<BlockTokenIdentifier> token;
+
+  /**
+   * The offset within the block to start reading at.
+   */
+  private long startOffset;
+
+  /**
+   * If false, we won't try to verify the block checksum.
+   */
+  private boolean verifyChecksum;
+
+  /**
+   * The name of this client.
+   */
+  private String clientName; 
+
+  /**
+   * The DataNode we're talking to.
+   */
+  private DatanodeInfo datanode;
+
+  /**
+   * StorageType of replica on DataNode.
+   */
+  private StorageType storageType;
+
+  /**
+   * If false, we won't try short-circuit local reads.
+   */
+  private boolean allowShortCircuitLocalReads;
+
+  /**
+   * The ClientContext to use for things like the PeerCache.
+   */
+  private ClientContext clientContext;
+
+  /**
+   * Number of bytes to read.  -1 indicates no limit.
+   */
+  private long length = -1;
+
+  /**
+   * Caching strategy to use when reading the block.
+   */
+  private CachingStrategy cachingStrategy;
+
+  /**
+   * Socket address to use to connect to peer.
+   */
+  private InetSocketAddress inetSocketAddress;
+
+  /**
+   * Remote peer factory to use to create a peer, if needed.
+   */
+  private RemotePeerFactory remotePeerFactory;
+
+  /**
+   * UserGroupInformation  to use for legacy block reader local objects, if needed.
+   */
+  private UserGroupInformation userGroupInformation;
+
+  /**
+   * Configuration to use for legacy block reader local objects, if needed.
+   */
+  private Configuration configuration;
+
+  /**
+   * Information about the domain socket path we should use to connect to the
+   * local peer-- or null if we haven't examined the local domain socket.
+   */
+  private DomainSocketFactory.PathInfo pathInfo;
+
+  /**
+   * The remaining number of times that we'll try to pull a socket out of the
+   * cache.
+   */
+  private int remainingCacheTries;
+
+  public BlockReaderFactory(DfsClientConf conf) {
+    this.conf = conf;
+    this.remainingCacheTries = conf.getNumCachedConnRetry();
+  }
+
+  public BlockReaderFactory setFileName(String fileName) {
+    this.fileName = fileName;
+    return this;
+  }
+
+  public BlockReaderFactory setBlock(ExtendedBlock block) {
+    this.block = block;
+    return this;
+  }
+
+  public BlockReaderFactory setBlockToken(Token<BlockTokenIdentifier> token) {
+    this.token = token;
+    return this;
+  }
+
+  public BlockReaderFactory setStartOffset(long startOffset) {
+    this.startOffset = startOffset;
+    return this;
+  }
+
+  public BlockReaderFactory setVerifyChecksum(boolean verifyChecksum) {
+    this.verifyChecksum = verifyChecksum;
+    return this;
+  }
+
+  public BlockReaderFactory setClientName(String clientName) {
+    this.clientName = clientName;
+    return this;
+  }
+
+  public BlockReaderFactory setDatanodeInfo(DatanodeInfo datanode) {
+    this.datanode = datanode;
+    return this;
+  }
+
+  public BlockReaderFactory setStorageType(StorageType storageType) {
+    this.storageType = storageType;
+    return this;
+  }
+
+  public BlockReaderFactory setAllowShortCircuitLocalReads(
+      boolean allowShortCircuitLocalReads) {
+    this.allowShortCircuitLocalReads = allowShortCircuitLocalReads;
+    return this;
+  }
+
+  public BlockReaderFactory setClientCacheContext(
+      ClientContext clientContext) {
+    this.clientContext = clientContext;
+    return this;
+  }
+
+  public BlockReaderFactory setLength(long length) {
+    this.length = length;
+    return this;
+  }
+
+  public BlockReaderFactory setCachingStrategy(
+      CachingStrategy cachingStrategy) {
+    this.cachingStrategy = cachingStrategy;
+    return this;
+  }
+
+  public BlockReaderFactory setInetSocketAddress (
+      InetSocketAddress inetSocketAddress) {
+    this.inetSocketAddress = inetSocketAddress;
+    return this;
+  }
+
+  public BlockReaderFactory setUserGroupInformation(
+      UserGroupInformation userGroupInformation) {
+    this.userGroupInformation = userGroupInformation;
+    return this;
+  }
+
+  public BlockReaderFactory setRemotePeerFactory(
+      RemotePeerFactory remotePeerFactory) {
+    this.remotePeerFactory = remotePeerFactory;
+    return this;
+  }
+
+  public BlockReaderFactory setConfiguration(
+      Configuration configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  @VisibleForTesting
+  public static void setFailureInjectorForTesting(FailureInjector injector) {
+    failureInjector = injector;
+  }
+
+  /**
+   * Build a BlockReader with the given options.
+   *
+   * This function will do the best it can to create a block reader that meets
+   * all of our requirements.  We prefer short-circuit block readers
+   * (BlockReaderLocal and BlockReaderLocalLegacy) over remote ones, since the
+   * former avoid the overhead of socket communication.  If short-circuit is
+   * unavailable, our next fallback is data transfer over UNIX domain sockets,
+   * if dfs.client.domain.socket.data.traffic has been enabled.  If that doesn't
+   * work, we will try to create a remote block reader that operates over TCP
+   * sockets.
+   *
+   * There are a few caches that are important here.
+   *
+   * The ShortCircuitCache stores file descriptor objects which have been passed
+   * from the DataNode. 
+   *
+   * The DomainSocketFactory stores information about UNIX domain socket paths
+   * that we not been able to use in the past, so that we don't waste time
+   * retrying them over and over.  (Like all the caches, it does have a timeout,
+   * though.)
+   *
+   * The PeerCache stores peers that we have used in the past.  If we can reuse
+   * one of these peers, we avoid the overhead of re-opening a socket.  However,
+   * if the socket has been timed out on the remote end, our attempt to reuse
+   * the socket may end with an IOException.  For that reason, we limit our
+   * attempts at socket reuse to dfs.client.cached.conn.retry times.  After
+   * that, we create new sockets.  This avoids the problem where a thread tries
+   * to talk to a peer that it hasn't talked to in a while, and has to clean out
+   * every entry in a socket cache full of stale entries.
+   *
+   * @return The new BlockReader.  We will not return null.
+   *
+   * @throws InvalidToken
+   *             If the block token was invalid.
+   *         InvalidEncryptionKeyException
+   *             If the encryption key was invalid.
+   *         Other IOException
+   *             If there was another problem.
+   */
+  public BlockReader build() throws IOException {
+    Preconditions.checkNotNull(configuration);
+    BlockReader reader = tryToCreateExternalBlockReader();
+    if (reader != null) {
+      return reader;
+    }
+    final ShortCircuitConf scConf = conf.getShortCircuitConf();
+    if (scConf.isShortCircuitLocalReads() && allowShortCircuitLocalReads) {
+      if (clientContext.getUseLegacyBlockReaderLocal()) {
+        reader = getLegacyBlockReaderLocal();
+        if (reader != null) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": returning new legacy block reader local.");
+          }
+          return reader;
+        }
+      } else {
+        reader = getBlockReaderLocal();
+        if (reader != null) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": returning new block reader local.");
+          }
+          return reader;
+        }
+      }
+    }
+    if (scConf.isDomainSocketDataTraffic()) {
+      reader = getRemoteBlockReaderFromDomain();
+      if (reader != null) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + ": returning new remote block reader using " +
+              "UNIX domain socket on " + pathInfo.getPath());
+        }
+        return reader;
+      }
+    }
+    Preconditions.checkState(!DFSInputStream.tcpReadsDisabledForTesting,
+        "TCP reads were disabled for testing, but we failed to " +
+        "do a non-TCP read.");
+    return getRemoteBlockReaderFromTcp();
+  }
+
+  private BlockReader tryToCreateExternalBlockReader() {
+    List<Class<? extends ReplicaAccessorBuilder>> clses =
+        conf.getReplicaAccessorBuilderClasses();
+    for (Class<? extends ReplicaAccessorBuilder> cls : clses) {
+      try {
+        ByteArrayDataOutput bado = ByteStreams.newDataOutput();
+        token.write(bado);
+        byte tokenBytes[] = bado.toByteArray();
+
+        Constructor<? extends ReplicaAccessorBuilder> ctor =
+            cls.getConstructor();
+        ReplicaAccessorBuilder builder = ctor.newInstance();
+        ReplicaAccessor accessor = builder.
+            setAllowShortCircuitReads(allowShortCircuitLocalReads).
+            setBlock(block.getBlockId(), block.getBlockPoolId()).
+            setGenerationStamp(block.getGenerationStamp()).
+            setBlockAccessToken(tokenBytes).
+            setClientName(clientName).
+            setConfiguration(configuration).
+            setFileName(fileName).
+            setVerifyChecksum(verifyChecksum).
+            setVisibleLength(length).
+            build();
+        if (accessor == null) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": No ReplicaAccessor created by " +
+                cls.getName());
+          }
+        } else {
+          return new ExternalBlockReader(accessor, length, startOffset);
+        }
+      } catch (Throwable t) {
+        LOG.warn("Failed to construct new object of type " +
+            cls.getName(), t);
+      }
+    }
+    return null;
+  }
+
+
+  /**
+   * Get {@link BlockReaderLocalLegacy} for short circuited local reads.
+   * This block reader implements the path-based style of local reads
+   * first introduced in HDFS-2246.
+   */
+  private BlockReader getLegacyBlockReaderLocal() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
+    }
+    if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
+            "the address " + inetSocketAddress + " is not local");
+      }
+      return null;
+    }
+    if (clientContext.getDisableLegacyBlockReaderLocal()) {
+        PerformanceAdvisory.LOG.debug("{}: can't construct " +
+            "BlockReaderLocalLegacy because " +
+            "disableLegacyBlockReaderLocal is set.", this);
+      return null;
+    }
+    IOException ioe;
+    try {
+      return BlockReaderLocalLegacy.newBlockReader(conf,
+          userGroupInformation, configuration, fileName, block, token,
+          datanode, startOffset, length, storageType);
+    } catch (RemoteException remoteException) {
+      ioe = remoteException.unwrapRemoteException(
+                InvalidToken.class, AccessControlException.class);
+    } catch (IOException e) {
+      ioe = e;
+    }
+    if ((!(ioe instanceof AccessControlException)) &&
+        isSecurityException(ioe)) {
+      // Handle security exceptions.
+      // We do not handle AccessControlException here, since
+      // BlockReaderLocalLegacy#newBlockReader uses that exception to indicate
+      // that the user is not in dfs.block.local-path-access.user, a condition
+      // which requires us to disable legacy SCR.
+      throw ioe;
+    }
+    LOG.warn(this + ": error creating legacy BlockReaderLocal.  " +
+        "Disabling legacy local reads.", ioe);
+    clientContext.setDisableLegacyBlockReaderLocal();
+    return null;
+  }
+
+  private BlockReader getBlockReaderLocal() throws InvalidToken {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": trying to construct a BlockReaderLocal " +
+          "for short-circuit reads.");
+    }
+    if (pathInfo == null) {
+      pathInfo = clientContext.getDomainSocketFactory()
+          .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
+    }
+    if (!pathInfo.getPathState().getUsableForShortCircuit()) {
+      PerformanceAdvisory.LOG.debug("{}: {} is not usable for short circuit; " +
+              "giving up on BlockReaderLocal.", this, pathInfo);
+      return null;
+    }
+    ShortCircuitCache cache = clientContext.getShortCircuitCache();
+    ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+    ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
+    InvalidToken exc = info.getInvalidTokenException();
+    if (exc != null) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": got InvalidToken exception while trying to " +
+            "construct BlockReaderLocal via " + pathInfo.getPath());
+      }
+      throw exc;
+    }
+    if (info.getReplica() == null) {
+      PerformanceAdvisory.LOG.debug("{}: failed to get " +
+          "ShortCircuitReplica. Cannot construct " +
+          "BlockReaderLocal via {}", this, pathInfo.getPath());
+      return null;
+    }
+    return new BlockReaderLocal.Builder(conf.getShortCircuitConf()).
+        setFilename(fileName).
+        setBlock(block).
+        setStartOffset(startOffset).
+        setShortCircuitReplica(info.getReplica()).
+        setVerifyChecksum(verifyChecksum).
+        setCachingStrategy(cachingStrategy).
+        setStorageType(storageType).
+        build();
+  }
+
+  /**
+   * Fetch a pair of short-circuit block descriptors from a local DataNode.
+   *
+   * @return    Null if we could not communicate with the datanode,
+   *            a new ShortCircuitReplicaInfo object otherwise.
+   *            ShortCircuitReplicaInfo objects may contain either an InvalidToken
+   *            exception, or a ShortCircuitReplica object ready to use.
+   */
+  @Override
+  public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
+    if (createShortCircuitReplicaInfoCallback != null) {
+      ShortCircuitReplicaInfo info =
+        createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
+      if (info != null) return info;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
+    }
+    BlockReaderPeer curPeer;
+    while (true) {
+      curPeer = nextDomainPeer();
+      if (curPeer == null) break;
+      if (curPeer.fromCache) remainingCacheTries--;
+      DomainPeer peer = (DomainPeer)curPeer.peer;
+      Slot slot = null;
+      ShortCircuitCache cache = clientContext.getShortCircuitCache();
+      try {
+        MutableBoolean usedPeer = new MutableBoolean(false);
+        slot = cache.allocShmSlot(datanode, peer, usedPeer,
+            new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
+            clientName);
+        if (usedPeer.booleanValue()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": allocShmSlot used up our previous socket " +
+              peer.getDomainSocket() + ".  Allocating a new one...");
+          }
+          curPeer = nextDomainPeer();
+          if (curPeer == null) break;
+          peer = (DomainPeer)curPeer.peer;
+        }
+        ShortCircuitReplicaInfo info = requestFileDescriptors(peer, slot);
+        clientContext.getPeerCache().put(datanode, peer);
+        return info;
+      } catch (IOException e) {
+        if (slot != null) {
+          cache.freeSlot(slot);
+        }
+        if (curPeer.fromCache) {
+          // Handle an I/O error we got when using a cached socket.
+          // These are considered less serious, because the socket may be stale.
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(this + ": closing stale domain peer " + peer, e);
+          }
+          IOUtilsClient.cleanup(LOG, peer);
+        } else {
+          // Handle an I/O error we got when using a newly created socket.
+          // We temporarily disable the domain socket path for a few minutes in
+          // this case, to prevent wasting more time on it.
+          LOG.warn(this + ": I/O error requesting file descriptors.  " + 
+              "Disabling domain socket " + peer.getDomainSocket(), e);
+          IOUtilsClient.cleanup(LOG, peer);
+          clientContext.getDomainSocketFactory()
+              .disableDomainSocketPath(pathInfo.getPath());
+          return null;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Request file descriptors from a DomainPeer.
+   *
+   * @param peer   The peer to use for communication.
+   * @param slot   If non-null, the shared memory slot to associate with the 
+   *               new ShortCircuitReplica.
+   * 
+   * @return  A ShortCircuitReplica object if we could communicate with the
+   *          datanode; null, otherwise. 
+   * @throws  IOException If we encountered an I/O exception while communicating
+   *          with the datanode.
+   */
+  private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
+          Slot slot) throws IOException {
+    ShortCircuitCache cache = clientContext.getShortCircuitCache();
+    final DataOutputStream out =
+        new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
+    SlotId slotId = slot == null ? null : slot.getSlotId();
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1,
+        failureInjector.getSupportsReceiptVerification());
+    DataInputStream in = new DataInputStream(peer.getInputStream());
+    BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+        PBHelperClient.vintPrefixed(in));
+    DomainSocket sock = peer.getDomainSocket();
+    failureInjector.injectRequestFileDescriptorsFailure();
+    switch (resp.getStatus()) {
+    case SUCCESS:
+      byte buf[] = new byte[1];
+      FileInputStream fis[] = new FileInputStream[2];
+      sock.recvFileInputStreams(fis, buf, 0, buf.length);
+      ShortCircuitReplica replica = null;
+      try {
+        ExtendedBlockId key =
+            new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
+          LOG.trace("Sending receipt verification byte for slot " + slot);
+          sock.getOutputStream().write(0);
+        }
+        replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
+            Time.monotonicNow(), slot);
+        return new ShortCircuitReplicaInfo(replica);
+      } catch (IOException e) {
+        // This indicates an error reading from disk, or a format error.  Since
+        // it's not a socket communication problem, we return null rather than
+        // throwing an exception.
+        LOG.warn(this + ": error creating ShortCircuitReplica.", e);
+        return null;
+      } finally {
+        if (replica == null) {
+          IOUtilsClient.cleanup(DFSClient.LOG, fis[0], fis[1]);
+        }
+      }
+    case ERROR_UNSUPPORTED:
+      if (!resp.hasShortCircuitAccessVersion()) {
+        LOG.warn("short-circuit read access is disabled for " +
+            "DataNode " + datanode + ".  reason: " + resp.getMessage());
+        clientContext.getDomainSocketFactory()
+            .disableShortCircuitForPath(pathInfo.getPath());
+      } else {
+        LOG.warn("short-circuit read access for the file " +
+            fileName + " is disabled for DataNode " + datanode +
+            ".  reason: " + resp.getMessage());
+      }
+      return null;
+    case ERROR_ACCESS_TOKEN:
+      String msg = "access control error while " +
+          "attempting to set up short-circuit access to " +
+          fileName + resp.getMessage();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(this + ":" + msg);
+      }
+      return new ShortCircuitReplicaInfo(new InvalidToken(msg));
+    default:
+      LOG.warn(this + ": unknown response code " + resp.getStatus() +
+          " while attempting to set up short-circuit access. " +
+          resp.getMessage());
+      clientContext.getDomainSocketFactory()
+          .disableShortCircuitForPath(pathInfo.getPath());
+      return null;
+    }
+  }
+
+  /**
+   * Get a RemoteBlockReader that communicates over a UNIX domain socket.
+   *
+   * @return The new BlockReader, or null if we failed to create the block
+   * reader.
+   *
+   * @throws InvalidToken    If the block token was invalid.
+   * Potentially other security-related execptions.
+   */
+  private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
+    if (pathInfo == null) {
+      pathInfo = clientContext.getDomainSocketFactory()
+          .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
+    }
+    if (!pathInfo.getPathState().getUsableForDataTransfer()) {
+      PerformanceAdvisory.LOG.debug("{}: not trying to create a " +
+          "remote block reader because the UNIX domain socket at {}" +
+           " is not usable.", this, pathInfo);
+      return null;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": trying to create a remote block reader from the " +
+          "UNIX domain socket at " + pathInfo.getPath());
+    }
+
+    while (true) {
+      BlockReaderPeer curPeer = nextDomainPeer();
+      if (curPeer == null) break;
+      if (curPeer.fromCache) remainingCacheTries--;
+      DomainPeer peer = (DomainPeer)curPeer.peer;
+      BlockReader blockReader = null;
+      try {
+        blockReader = getRemoteBlockReader(peer);
+        return blockReader;
+      } catch (IOException ioe) {
+        IOUtilsClient.cleanup(LOG, peer);
+        if (isSecurityException(ioe)) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": got security exception while constructing " +
+                "a remote block reader from the unix domain socket at " +
+                pathInfo.getPath(), ioe);
+          }
+          throw ioe;
+        }
+        if (curPeer.fromCache) {
+          // Handle an I/O error we got when using a cached peer.  These are
+          // considered less serious, because the underlying socket may be stale.
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Closed potentially stale domain peer " + peer, ioe);
+          }
+        } else {
+          // Handle an I/O error we got when using a newly created domain peer.
+          // We temporarily disable the domain socket path for a few minutes in
+          // this case, to prevent wasting more time on it.
+          LOG.warn("I/O error constructing remote block reader.  Disabling " +
+              "domain socket " + peer.getDomainSocket(), ioe);
+          clientContext.getDomainSocketFactory()
+              .disableDomainSocketPath(pathInfo.getPath());
+          return null;
+        }
+      } finally {
+        if (blockReader == null) {
+          IOUtilsClient.cleanup(LOG, peer);
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get a RemoteBlockReader that communicates over a TCP socket.
+   *
+   * @return The new BlockReader.  We will not return null, but instead throw
+   *         an exception if this fails.
+   *
+   * @throws InvalidToken
+   *             If the block token was invalid.
+   *         InvalidEncryptionKeyException
+   *             If the encryption key was invalid.
+   *         Other IOException
+   *             If there was another problem.
+   */
+  private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": trying to create a remote block reader from a " +
+          "TCP socket");
+    }
+    BlockReader blockReader = null;
+    while (true) {
+      BlockReaderPeer curPeer = null;
+      Peer peer = null;
+      try {
+        curPeer = nextTcpPeer();
+        if (curPeer.fromCache) remainingCacheTries--;
+        peer = curPeer.peer;
+        blockReader = getRemoteBlockReader(peer);
+        return blockReader;
+      } catch (IOException ioe) {
+        if (isSecurityException(ioe)) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": got security exception while constructing " +
+                "a remote block reader from " + peer, ioe);
+          }
+          throw ioe;
+        }
+        if ((curPeer != null) && curPeer.fromCache) {
+          // Handle an I/O error we got when using a cached peer.  These are
+          // considered less serious, because the underlying socket may be
+          // stale.
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Closed potentially stale remote peer " + peer, ioe);
+          }
+        } else {
+          // Handle an I/O error we got when using a newly created peer.
+          LOG.warn("I/O error constructing remote block reader.", ioe);
+          throw ioe;
+        }
+      } finally {
+        if (blockReader == null) {
+          IOUtilsClient.cleanup(LOG, peer);
+        }
+      }
+    }
+  }
+
+  public static class BlockReaderPeer {
+    final Peer peer;
+    final boolean fromCache;
+    
+    BlockReaderPeer(Peer peer, boolean fromCache) {
+      this.peer = peer;
+      this.fromCache = fromCache;
+    }
+  }
+
+  /**
+   * Get the next DomainPeer-- either from the cache or by creating it.
+   *
+   * @return the next DomainPeer, or null if we could not construct one.
+   */
+  private BlockReaderPeer nextDomainPeer() {
+    if (remainingCacheTries > 0) {
+      Peer peer = clientContext.getPeerCache().get(datanode, true);
+      if (peer != null) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("nextDomainPeer: reusing existing peer " + peer);
+        }
+        return new BlockReaderPeer(peer, true);
+      }
+    }
+    DomainSocket sock = clientContext.getDomainSocketFactory().
+        createSocket(pathInfo, conf.getSocketTimeout());
+    if (sock == null) return null;
+    return new BlockReaderPeer(new DomainPeer(sock), false);
+  }
+
+  /**
+   * Get the next TCP-based peer-- either from the cache or by creating it.
+   *
+   * @return the next Peer, or null if we could not construct one.
+   *
+   * @throws IOException  If there was an error while constructing the peer
+   *                      (such as an InvalidEncryptionKeyException)
+   */
+  private BlockReaderPeer nextTcpPeer() throws IOException {
+    if (remainingCacheTries > 0) {
+      Peer peer = clientContext.getPeerCache().get(datanode, false);
+      if (peer != null) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("nextTcpPeer: reusing existing peer " + peer);
+        }
+        return new BlockReaderPeer(peer, true);
+      }
+    }
+    try {
+      Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
+        datanode);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("nextTcpPeer: created newConnectedPeer " + peer);
+      }
+      return new BlockReaderPeer(peer, false);
+    } catch (IOException e) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("nextTcpPeer: failed to create newConnectedPeer " +
+                  "connected to " + datanode);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Determine if an exception is security-related.
+   *
+   * We need to handle these exceptions differently than other IOExceptions.
+   * They don't indicate a communication problem.  Instead, they mean that there
+   * is some action the client needs to take, such as refetching block tokens,
+   * renewing encryption keys, etc.
+   *
+   * @param ioe    The exception
+   * @return       True only if the exception is security-related.
+   */
+  private static boolean isSecurityException(IOException ioe) {
+    return (ioe instanceof InvalidToken) ||
+            (ioe instanceof InvalidEncryptionKeyException) ||
+            (ioe instanceof InvalidBlockTokenException) ||
+            (ioe instanceof AccessControlException);
+  }
+
+  @SuppressWarnings("deprecation")
+  private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
+    if (conf.getShortCircuitConf().isUseLegacyBlockReader()) {
+      return RemoteBlockReader.newBlockReader(fileName,
+          block, token, startOffset, length, conf.getIoBufferSize(),
+          verifyChecksum, clientName, peer, datanode,
+          clientContext.getPeerCache(), cachingStrategy);
+    } else {
+      return RemoteBlockReader2.newBlockReader(
+          fileName, block, token, startOffset, length,
+          verifyChecksum, clientName, peer, datanode,
+          clientContext.getPeerCache(), cachingStrategy);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "BlockReaderFactory(fileName=" + fileName + ", block=" + block + ")";
+  }
+
+  /**
+   * File name to print when accessing a block directly (from servlets)
+   * @param s Address of the block location
+   * @param poolId Block pool ID of the block
+   * @param blockId Block ID of the block
+   * @return string that has a file name for debug purposes
+   */
+  public static String getFileName(final InetSocketAddress s,
+      final String poolId, final long blockId) {
+    return s.toString() + ":" + poolId + ":" + blockId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
new file mode 100644
index 0000000..807ede8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
@@ -0,0 +1,369 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.HdfsVolumeId;
+import org.apache.hadoop.fs.VolumeId;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.token.Token;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class BlockStorageLocationUtil {
+  
+  static final Logger LOG = LoggerFactory
+      .getLogger(BlockStorageLocationUtil.class);
+  
+  /**
+   * Create a list of {@link VolumeBlockLocationCallable} corresponding to a set
+   * of datanodes and blocks. The blocks must all correspond to the same
+   * block pool.
+   * 
+   * @param datanodeBlocks
+   *          Map of datanodes to block replicas at each datanode
+   * @return callables Used to query each datanode for location information on
+   *         the block replicas at the datanode
+   */
+  private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables(
+      Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
+      int timeout, boolean connectToDnViaHostname, Span parent) {
+    
+    if (datanodeBlocks.isEmpty()) {
+      return Lists.newArrayList();
+    }
+    
+    // Construct the callables, one per datanode
+    List<VolumeBlockLocationCallable> callables = 
+        new ArrayList<VolumeBlockLocationCallable>();
+    for (Map.Entry<DatanodeInfo, List<LocatedBlock>> entry : datanodeBlocks
+        .entrySet()) {
+      // Construct RPC parameters
+      DatanodeInfo datanode = entry.getKey();
+      List<LocatedBlock> locatedBlocks = entry.getValue();
+      if (locatedBlocks.isEmpty()) {
+        continue;
+      }
+      
+      // Ensure that the blocks all are from the same block pool.
+      String poolId = locatedBlocks.get(0).getBlock().getBlockPoolId();
+      for (LocatedBlock lb : locatedBlocks) {
+        if (!poolId.equals(lb.getBlock().getBlockPoolId())) {
+          throw new IllegalArgumentException(
+              "All blocks to be queried must be in the same block pool: " +
+              locatedBlocks.get(0).getBlock() + " and " + lb +
+              " are from different pools.");
+        }
+      }
+      
+      long[] blockIds = new long[locatedBlocks.size()];
+      int i = 0;
+      List<Token<BlockTokenIdentifier>> dnTokens = 
+          new ArrayList<Token<BlockTokenIdentifier>>(
+          locatedBlocks.size());
+      for (LocatedBlock b : locatedBlocks) {
+        blockIds[i++] = b.getBlock().getBlockId();
+        dnTokens.add(b.getBlockToken());
+      }
+      VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable(
+          conf, datanode, poolId, blockIds, dnTokens, timeout, 
+          connectToDnViaHostname, parent);
+      callables.add(callable);
+    }
+    return callables;
+  }
+  
+  /**
+   * Queries datanodes for the blocks specified in <code>datanodeBlocks</code>,
+   * making one RPC to each datanode. These RPCs are made in parallel using a
+   * threadpool.
+   * 
+   * @param datanodeBlocks
+   *          Map of datanodes to the blocks present on the DN
+   * @return metadatas Map of datanodes to block metadata of the DN
+   * @throws InvalidBlockTokenException
+   *           if client does not have read access on a requested block
+   */
+  static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
+      Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
+      int poolsize, int timeoutMs, boolean connectToDnViaHostname)
+        throws InvalidBlockTokenException {
+
+    List<VolumeBlockLocationCallable> callables = 
+        createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs, 
+            connectToDnViaHostname, Trace.currentSpan());
+    
+    // Use a thread pool to execute the Callables in parallel
+    List<Future<HdfsBlocksMetadata>> futures = 
+        new ArrayList<Future<HdfsBlocksMetadata>>();
+    ExecutorService executor = new ScheduledThreadPoolExecutor(poolsize);
+    try {
+      futures = executor.invokeAll(callables, timeoutMs,
+          TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      // Swallow the exception here, because we can return partial results
+    }
+    executor.shutdown();
+    
+    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas =
+        Maps.newHashMapWithExpectedSize(datanodeBlocks.size());
+    // Fill in metadatas with results from DN RPCs, where possible
+    for (int i = 0; i < futures.size(); i++) {
+      VolumeBlockLocationCallable callable = callables.get(i);
+      DatanodeInfo datanode = callable.getDatanodeInfo();
+      Future<HdfsBlocksMetadata> future = futures.get(i);
+      try {
+        HdfsBlocksMetadata metadata = future.get();
+        metadatas.put(callable.getDatanodeInfo(), metadata);
+      } catch (CancellationException e) {
+        LOG.info("Cancelled while waiting for datanode "
+            + datanode.getIpcAddr(false) + ": " + e.toString());
+      } catch (ExecutionException e) {
+        Throwable t = e.getCause();
+        if (t instanceof InvalidBlockTokenException) {
+          LOG.warn("Invalid access token when trying to retrieve "
+              + "information from datanode " + datanode.getIpcAddr(false));
+          throw (InvalidBlockTokenException) t;
+        }
+        else if (t instanceof UnsupportedOperationException) {
+          LOG.info("Datanode " + datanode.getIpcAddr(false) + " does not support"
+              + " required #getHdfsBlocksMetadata() API");
+          throw (UnsupportedOperationException) t;
+        } else {
+          LOG.info("Failed to query block locations on datanode " +
+              datanode.getIpcAddr(false) + ": " + t);
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Could not fetch information from datanode", t);
+        }
+      } catch (InterruptedException e) {
+        // Shouldn't happen, because invokeAll waits for all Futures to be ready
+        LOG.info("Interrupted while fetching HdfsBlocksMetadata");
+      }
+    }
+    
+    return metadatas;
+  }
+  
+  /**
+   * Group the per-replica {@link VolumeId} info returned from
+   * {@link DFSClient#queryDatanodesForHdfsBlocksMetadata(Map)} to be
+   * associated
+   * with the corresponding {@link LocatedBlock}.
+   * 
+   * @param blocks
+   *          Original LocatedBlock array
+   * @param metadatas
+   *          VolumeId information for the replicas on each datanode
+   * @return blockVolumeIds per-replica VolumeId information associated with the
+   *         parent LocatedBlock
+   */
+  static Map<LocatedBlock, List<VolumeId>> associateVolumeIdsWithBlocks(
+      List<LocatedBlock> blocks,
+      Map<DatanodeInfo, HdfsBlocksMetadata> metadatas) {
+    
+    // Initialize mapping of ExtendedBlock to LocatedBlock. 
+    // Used to associate results from DN RPCs to the parent LocatedBlock
+    Map<Long, LocatedBlock> blockIdToLocBlock = 
+        new HashMap<Long, LocatedBlock>();
+    for (LocatedBlock b : blocks) {
+      blockIdToLocBlock.put(b.getBlock().getBlockId(), b);
+    }
+    
+    // Initialize the mapping of blocks -> list of VolumeIds, one per replica
+    // This is filled out with real values from the DN RPCs
+    Map<LocatedBlock, List<VolumeId>> blockVolumeIds = 
+        new HashMap<LocatedBlock, List<VolumeId>>();
+    for (LocatedBlock b : blocks) {
+      ArrayList<VolumeId> l = new ArrayList<VolumeId>(b.getLocations().length);
+      for (int i = 0; i < b.getLocations().length; i++) {
+        l.add(null);
+      }
+      blockVolumeIds.put(b, l);
+    }
+    
+    // Iterate through the list of metadatas (one per datanode). 
+    // For each metadata, if it's valid, insert its volume location information 
+    // into the Map returned to the caller 
+    for (Map.Entry<DatanodeInfo, HdfsBlocksMetadata> entry : metadatas.entrySet()) {
+      DatanodeInfo datanode = entry.getKey();
+      HdfsBlocksMetadata metadata = entry.getValue();
+      // Check if metadata is valid
+      if (metadata == null) {
+        continue;
+      }
+      long[] metaBlockIds = metadata.getBlockIds();
+      List<byte[]> metaVolumeIds = metadata.getVolumeIds();
+      List<Integer> metaVolumeIndexes = metadata.getVolumeIndexes();
+      // Add VolumeId for each replica in the HdfsBlocksMetadata
+      for (int j = 0; j < metaBlockIds.length; j++) {
+        int volumeIndex = metaVolumeIndexes.get(j);
+        long blockId = metaBlockIds[j];
+        // Skip if block wasn't found, or not a valid index into metaVolumeIds
+        // Also skip if the DN responded with a block we didn't ask for
+        if (volumeIndex == Integer.MAX_VALUE
+            || volumeIndex >= metaVolumeIds.size()
+            || !blockIdToLocBlock.containsKey(blockId)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("No data for block " + blockId);
+          }
+          continue;
+        }
+        // Get the VolumeId by indexing into the list of VolumeIds
+        // provided by the datanode
+        byte[] volumeId = metaVolumeIds.get(volumeIndex);
+        HdfsVolumeId id = new HdfsVolumeId(volumeId);
+        // Find out which index we are in the LocatedBlock's replicas
+        LocatedBlock locBlock = blockIdToLocBlock.get(blockId);
+        DatanodeInfo[] dnInfos = locBlock.getLocations();
+        int index = -1;
+        for (int k = 0; k < dnInfos.length; k++) {
+          if (dnInfos[k].equals(datanode)) {
+            index = k;
+            break;
+          }
+        }
+        if (index < 0) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Datanode responded with a block volume id we did" +
+                " not request, omitting.");
+          }
+          continue;
+        }
+        // Place VolumeId at the same index as the DN's index in the list of
+        // replicas
+        List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
+        volumeIds.set(index, id);
+      }
+    }
+    return blockVolumeIds;
+  }
+
+  /**
+   * Helper method to combine a list of {@link LocatedBlock} with associated
+   * {@link VolumeId} information to form a list of {@link BlockStorageLocation}
+   * .
+   */
+  static BlockStorageLocation[] convertToVolumeBlockLocations(
+      List<LocatedBlock> blocks, 
+      Map<LocatedBlock, List<VolumeId>> blockVolumeIds) throws IOException {
+    // Construct the final return value of VolumeBlockLocation[]
+    BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks);
+    List<BlockStorageLocation> volumeBlockLocs = 
+        new ArrayList<BlockStorageLocation>(locations.length);
+    for (int i = 0; i < locations.length; i++) {
+      LocatedBlock locBlock = blocks.get(i);
+      List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
+      BlockStorageLocation bsLoc = new BlockStorageLocation(locations[i], 
+          volumeIds.toArray(new VolumeId[0]));
+      volumeBlockLocs.add(bsLoc);
+    }
+    return volumeBlockLocs.toArray(new BlockStorageLocation[] {});
+  }
+  
+  /**
+   * Callable that sets up an RPC proxy to a datanode and queries it for
+   * volume location information for a list of ExtendedBlocks. 
+   */
+  private static class VolumeBlockLocationCallable implements 
+    Callable<HdfsBlocksMetadata> {
+    
+    private final Configuration configuration;
+    private final int timeout;
+    private final DatanodeInfo datanode;
+    private final String poolId;
+    private final long[] blockIds;
+    private final List<Token<BlockTokenIdentifier>> dnTokens;
+    private final boolean connectToDnViaHostname;
+    private final Span parentSpan;
+    
+    VolumeBlockLocationCallable(Configuration configuration,
+        DatanodeInfo datanode, String poolId, long []blockIds,
+        List<Token<BlockTokenIdentifier>> dnTokens, int timeout, 
+        boolean connectToDnViaHostname, Span parentSpan) {
+      this.configuration = configuration;
+      this.timeout = timeout;
+      this.datanode = datanode;
+      this.poolId = poolId;
+      this.blockIds = blockIds;
+      this.dnTokens = dnTokens;
+      this.connectToDnViaHostname = connectToDnViaHostname;
+      this.parentSpan = parentSpan;
+    }
+    
+    public DatanodeInfo getDatanodeInfo() {
+      return datanode;
+    }
+
+    @Override
+    public HdfsBlocksMetadata call() throws Exception {
+      HdfsBlocksMetadata metadata = null;
+      // Create the RPC proxy and make the RPC
+      ClientDatanodeProtocol cdp = null;
+      TraceScope scope =
+          Trace.startSpan("getHdfsBlocksMetadata", parentSpan);
+      try {
+        cdp = DFSUtilClient.createClientDatanodeProtocolProxy(
+            datanode, configuration,
+            timeout, connectToDnViaHostname);
+        metadata = cdp.getHdfsBlocksMetadata(poolId, blockIds, dnTokens);
+      } catch (IOException e) {
+        // Bubble this up to the caller, handle with the Future
+        throw e;
+      } finally {
+        scope.close();
+        if (cdp != null) {
+          RPC.stopProxy(cdp);
+        }
+      }
+      return metadata;
+    }
+  }
+}