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 zh...@apache.org on 2015/09/30 17:41:01 UTC

[01/58] [abbrv] 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/HDFS-7285 6419900ac -> 6c17d3152


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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 be1a9ef..ff2d762 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
@@ -47,6 +47,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 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;
@@ -877,7 +878,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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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 7c935d3..1bb4485 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
@@ -108,6 +108,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+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;
@@ -2156,7 +2157,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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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 a586a7c..68c951a 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/bf37d3d8/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 32eb724..466c7e9 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
@@ -57,7 +57,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.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -263,8 +263,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,
@@ -956,7 +956,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/bf37d3d8/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 2a4c191..9b467ea 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/bf37d3d8/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/bf37d3d8/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 afecf99..4af2021 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;
@@ -115,7 +115,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/bf37d3d8/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/bf37d3d8/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 a7e80ca..bce8b64 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
@@ -982,7 +982,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/bf37d3d8/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/bf37d3d8/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 cfc317f..62d1a3e 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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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 43650a8..2789e33 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
@@ -292,9 +292,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
@@ -368,9 +367,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


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

Posted by zh...@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/bf37d3d8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bf37d3d8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bf37d3d8

Branch: refs/heads/HDFS-7285
Commit: bf37d3d80e5179dea27e5bd5aea804a38aa9934c
Parents: 861b52d
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:08:25 2015 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   24 +
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 +
 .../hadoop/hdfs/BlockMissingException.java      |   65 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  893 +++++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3144 ++++++++++++++++++
 .../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 |  918 +++++
 .../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 -
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 -
 .../hadoop/hdfs/BlockMissingException.java      |   65 -
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  892 -----
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3141 -----------------
 .../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 |  918 -----
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  345 --
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   24 -
 .../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 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |   10 +-
 81 files changed, 11544 insertions(+), 11489 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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;
+  }
+}


[35/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 0000000,ea86d2d..18cc124
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@@ -1,0 -1,2259 +1,2335 @@@
+ /**
+  * 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.net.InetSocketAddress;
+ import java.net.URI;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.EnumSet;
+ import java.util.List;
+ import java.util.Map;
+ 
+ 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.BlockStoragePolicySpi;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FSDataInputStream;
+ import org.apache.hadoop.fs.FSDataOutputStream;
+ import org.apache.hadoop.fs.FSLinkResolver;
+ import org.apache.hadoop.fs.FileAlreadyExistsException;
+ import org.apache.hadoop.fs.FileChecksum;
+ import org.apache.hadoop.fs.FileStatus;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.FileSystemLinkResolver;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.FsStatus;
+ import org.apache.hadoop.fs.LocatedFileStatus;
+ import org.apache.hadoop.fs.Options;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.Options.ChecksumOpt;
+ import org.apache.hadoop.fs.ParentNotDirectoryException;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.PathFilter;
+ import org.apache.hadoop.fs.RemoteIterator;
+ import org.apache.hadoop.fs.UnresolvedLinkException;
+ import org.apache.hadoop.fs.UnsupportedFileSystemException;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+ import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
+ 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.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ 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.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.HdfsLocatedFileStatus;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.io.Text;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+ import org.apache.hadoop.net.NetUtils;
+ import org.apache.hadoop.security.AccessControlException;
+ import org.apache.hadoop.security.Credentials;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.Progressable;
+ import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Preconditions;
+ 
+ 
+ /****************************************************************
+  * Implementation of the abstract FileSystem for the DFS system.
+  * This object is the way end-user code interacts with a Hadoop
+  * DistributedFileSystem.
+  *
+  *****************************************************************/
+ @InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
+ @InterfaceStability.Unstable
+ public class DistributedFileSystem extends FileSystem {
+   private Path workingDir;
+   private URI uri;
+   private String homeDirPrefix =
+       HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT;
+ 
+   DFSClient dfs;
+   private boolean verifyChecksum = true;
+   
+   static{
+     HdfsConfigurationLoader.init();
+   }
+ 
+   public DistributedFileSystem() {
+   }
+ 
+   /**
+    * Return the protocol scheme for the FileSystem.
+    * <p/>
+    *
+    * @return <code>hdfs</code>
+    */
+   @Override
+   public String getScheme() {
+     return HdfsConstants.HDFS_URI_SCHEME;
+   }
+ 
+   @Override
+   public URI getUri() { return uri; }
+ 
+   @Override
+   public void initialize(URI uri, Configuration conf) throws IOException {
+     super.initialize(uri, conf);
+     setConf(conf);
+ 
+     String host = uri.getHost();
+     if (host == null) {
+       throw new IOException("Incomplete HDFS URI, no host: "+ uri);
+     }
+     homeDirPrefix = conf.get(
+         HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_KEY,
+         HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT);
+ 
+     this.dfs = new DFSClient(uri, conf, statistics);
+     this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
+     this.workingDir = getHomeDirectory();
+   }
+ 
+   @Override
+   public Path getWorkingDirectory() {
+     return workingDir;
+   }
+ 
+   @Override
+   public long getDefaultBlockSize() {
+     return dfs.getConf().getDefaultBlockSize();
+   }
+ 
+   @Override
+   public short getDefaultReplication() {
+     return dfs.getConf().getDefaultReplication();
+   }
+ 
+   @Override
+   public void setWorkingDirectory(Path dir) {
+     String result = fixRelativePart(dir).toUri().getPath();
+     if (!DFSUtilClient.isValidName(result)) {
+       throw new IllegalArgumentException("Invalid DFS directory name " + 
+                                          result);
+     }
+     workingDir = fixRelativePart(dir);
+   }
+ 
+   @Override
+   public Path getHomeDirectory() {
+     return makeQualified(new Path(homeDirPrefix + "/"
+         + dfs.ugi.getShortUserName()));
+   }
+ 
+   /**
+    * Checks that the passed URI belongs to this filesystem and returns
+    * just the path component. Expects a URI with an absolute path.
+    * 
+    * @param file URI with absolute path
+    * @return path component of {file}
+    * @throws IllegalArgumentException if URI does not belong to this DFS
+    */
+   private String getPathName(Path file) {
+     checkPath(file);
+     String result = file.toUri().getPath();
+     if (!DFSUtilClient.isValidName(result)) {
+       throw new IllegalArgumentException("Pathname " + result + " from " +
+                                          file+" is not a valid DFS filename.");
+     }
+     return result;
+   }
+   
+   @Override
+   public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+       long len) throws IOException {
+     if (file == null) {
+       return null;
+     }
+     return getFileBlockLocations(file.getPath(), start, len);
+   }
+   
+   @Override
+   public BlockLocation[] getFileBlockLocations(Path p, 
+       final long start, final long len) throws IOException {
+     statistics.incrementReadOps(1);
+     final Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<BlockLocation[]>() {
+       @Override
+       public BlockLocation[] doCall(final Path p) throws IOException {
+         return dfs.getBlockLocations(getPathName(p), start, len);
+       }
+       @Override
+       public BlockLocation[] next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getFileBlockLocations(p, start, len);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setVerifyChecksum(boolean verifyChecksum) {
+     this.verifyChecksum = verifyChecksum;
+   }
+ 
+   /** 
+    * Start the lease recovery of a file
+    *
+    * @param f a file
+    * @return true if the file is already closed
+    * @throws IOException if an error occurs
+    */
+   public boolean recoverLease(final Path f) throws IOException {
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.recoverLease(getPathName(p));
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.recoverLease(p);
+         }
+         throw new UnsupportedOperationException("Cannot recoverLease through" +
+             " a symlink to a non-DistributedFileSystem: " + f + " -> " + p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FSDataInputStream open(Path f, final int bufferSize)
+       throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataInputStream>() {
+       @Override
+       public FSDataInputStream doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         final DFSInputStream dfsis =
+           dfs.open(getPathName(p), bufferSize, verifyChecksum);
+         return dfs.createWrappedInputStream(dfsis);
+       }
+       @Override
+       public FSDataInputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.open(p, bufferSize);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FSDataOutputStream append(Path f, final int bufferSize,
+       final Progressable progress) throws IOException {
+     return append(f, EnumSet.of(CreateFlag.APPEND), bufferSize, progress);
+   }
+ 
+   /**
+    * Append to an existing file (optional operation).
+    * 
+    * @param f the existing file to be appended.
+    * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
+    *          to be present.
+    * @param bufferSize the size of the buffer to be used.
+    * @param progress for reporting progress if it is not null.
+    * @return Returns instance of {@link FSDataOutputStream}
+    * @throws IOException
+    */
+   public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
+       final int bufferSize, final Progressable progress) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p)
+           throws IOException {
+         return dfs.append(getPathName(p), bufferSize, flag, progress,
+             statistics);
+       }
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.append(p, bufferSize);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Append to an existing file (optional operation).
+    * 
+    * @param f the existing file to be appended.
+    * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
+    *          to be present.
+    * @param bufferSize the size of the buffer to be used.
+    * @param progress for reporting progress if it is not null.
+    * @param favoredNodes Favored nodes for new blocks
+    * @return Returns instance of {@link FSDataOutputStream}
+    * @throws IOException
+    */
+   public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
+       final int bufferSize, final Progressable progress,
+       final InetSocketAddress[] favoredNodes) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p)
+           throws IOException {
+         return dfs.append(getPathName(p), bufferSize, flag, progress,
+             statistics, favoredNodes);
+       }
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.append(p, bufferSize);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FSDataOutputStream create(Path f, FsPermission permission,
+       boolean overwrite, int bufferSize, short replication, long blockSize,
+       Progressable progress) throws IOException {
+     return this.create(f, permission,
+         overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+             : EnumSet.of(CreateFlag.CREATE), bufferSize, replication,
+         blockSize, progress, null);
+   }
+ 
+   /**
+    * Same as  
+    * {@link #create(Path, FsPermission, boolean, int, short, long, 
+    * Progressable)} 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. And with favored nodes, blocks will be pinned
+    * on the datanodes to prevent balancing move the block. HDFS could move the
+    * blocks during replication, to move the blocks from favored nodes. A value
+    * of null means no favored nodes for this create
+    */
+   public HdfsDataOutputStream create(final Path f,
+       final FsPermission permission, final boolean overwrite,
+       final int bufferSize, final short replication, final long blockSize,
+       final Progressable progress, final InetSocketAddress[] favoredNodes)
+           throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<HdfsDataOutputStream>() {
+       @Override
+       public HdfsDataOutputStream doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         final DFSOutputStream out = dfs.create(getPathName(f), permission,
+             overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+                 : EnumSet.of(CreateFlag.CREATE),
+             true, replication, blockSize, progress, bufferSize, null,
+             favoredNodes);
+         return dfs.createWrappedOutputStream(out, statistics);
+       }
+       @Override
+       public HdfsDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.create(p, permission, overwrite, bufferSize, replication,
+               blockSize, progress, favoredNodes);
+         }
+         throw new UnsupportedOperationException("Cannot create with" +
+             " favoredNodes through a symlink to a non-DistributedFileSystem: "
+             + f + " -> " + p);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public FSDataOutputStream create(final Path f, final FsPermission permission,
+     final EnumSet<CreateFlag> cflags, final int bufferSize,
+     final short replication, final long blockSize, final Progressable progress,
+     final ChecksumOpt checksumOpt) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
+                 cflags, replication, blockSize, progress, bufferSize,
+                 checksumOpt);
+         return dfs.createWrappedOutputStream(dfsos, statistics);
+       }
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.create(p, permission, cflags, bufferSize,
+             replication, blockSize, progress, checksumOpt);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   protected HdfsDataOutputStream primitiveCreate(Path f,
+     FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
+     short replication, long blockSize, Progressable progress,
+     ChecksumOpt checksumOpt) throws IOException {
+     statistics.incrementWriteOps(1);
+     final DFSOutputStream dfsos = dfs.primitiveCreate(
+       getPathName(fixRelativePart(f)),
+       absolutePermission, flag, true, replication, blockSize,
+       progress, bufferSize, checksumOpt);
+     return dfs.createWrappedOutputStream(dfsos, statistics);
+   }
+ 
+   /**
+    * Same as create(), except fails if parent directory doesn't already exist.
+    */
+   @Override
+   public FSDataOutputStream createNonRecursive(final Path f,
+       final FsPermission permission, final EnumSet<CreateFlag> flag,
+       final int bufferSize, final short replication, final long blockSize,
+       final Progressable progress) throws IOException {
+     statistics.incrementWriteOps(1);
+     if (flag.contains(CreateFlag.OVERWRITE)) {
+       flag.add(CreateFlag.CREATE);
+     }
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
+           flag, false, replication, blockSize, progress, bufferSize, null);
+         return dfs.createWrappedOutputStream(dfsos, statistics);
+       }
+ 
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.createNonRecursive(p, permission, flag, bufferSize,
+             replication, blockSize, progress);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public boolean setReplication(Path src, 
+                                 final short replication
+                                ) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(src);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.setReplication(getPathName(p), replication);
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.setReplication(p, replication);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Set the source path to the specified storage policy.
+    *
+    * @param src The source path referring to either a directory or a file.
+    * @param policyName The name of the storage policy.
+    */
+   @Override
+   public void setStoragePolicy(final Path src, final String policyName)
+       throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(src);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setStoragePolicy(getPathName(p), policyName);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setStoragePolicy(p, policyName);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(path);
+ 
+     return new FileSystemLinkResolver<BlockStoragePolicySpi>() {
+       @Override
+       public BlockStoragePolicySpi doCall(final Path p) throws IOException {
+         return getClient().getStoragePolicy(getPathName(p));
+       }
+ 
+       @Override
+       public BlockStoragePolicySpi next(final FileSystem fs, final Path p)
+           throws IOException, UnresolvedLinkException {
+         return fs.getStoragePolicy(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public Collection<BlockStoragePolicy> getAllStoragePolicies()
+       throws IOException {
+     return Arrays.asList(dfs.getStoragePolicies());
+   }
+ 
+   /**
+    * Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
+    * @return
+    * @throws IOException
+    */
+   @Deprecated
+   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+     statistics.incrementReadOps(1);
+     return dfs.getStoragePolicies();
+   }
+ 
+   /**
+    * Move blocks from srcs to trg and delete srcs afterwards.
+    * The file block sizes must be the same.
+    * 
+    * @param trg existing file to append to
+    * @param psrcs list of files (same block size, same replication)
+    * @throws IOException
+    */
+   @Override
+   public void concat(Path trg, Path [] psrcs) throws IOException {
+     statistics.incrementWriteOps(1);
+     // Make target absolute
+     Path absF = fixRelativePart(trg);
+     // Make all srcs absolute
+     Path[] srcs = new Path[psrcs.length];
+     for (int i=0; i<psrcs.length; i++) {
+       srcs[i] = fixRelativePart(psrcs[i]);
+     }
+     // Try the concat without resolving any links
+     String[] srcsStr = new String[psrcs.length];
+     try {
+       for (int i=0; i<psrcs.length; i++) {
+         srcsStr[i] = getPathName(srcs[i]);
+       }
+       dfs.concat(getPathName(trg), srcsStr);
+     } catch (UnresolvedLinkException e) {
+       // Exception could be from trg or any src.
+       // Fully resolve trg and srcs. Fail if any of them are a symlink.
+       FileStatus stat = getFileLinkStatus(absF);
+       if (stat.isSymlink()) {
+         throw new IOException("Cannot concat with a symlink target: "
+             + trg + " -> " + stat.getPath());
+       }
+       absF = fixRelativePart(stat.getPath());
+       for (int i=0; i<psrcs.length; i++) {
+         stat = getFileLinkStatus(srcs[i]);
+         if (stat.isSymlink()) {
+           throw new IOException("Cannot concat with a symlink src: "
+               + psrcs[i] + " -> " + stat.getPath());
+         }
+         srcs[i] = fixRelativePart(stat.getPath());
+       }
+       // Try concat again. Can still race with another symlink.
+       for (int i=0; i<psrcs.length; i++) {
+         srcsStr[i] = getPathName(srcs[i]);
+       }
+       dfs.concat(getPathName(absF), srcsStr);
+     }
+   }
+ 
+   
+   @SuppressWarnings("deprecation")
+   @Override
+   public boolean rename(Path src, Path dst) throws IOException {
+     statistics.incrementWriteOps(1);
+ 
+     final Path absSrc = fixRelativePart(src);
+     final Path absDst = fixRelativePart(dst);
+ 
+     // Try the rename without resolving first
+     try {
+       return dfs.rename(getPathName(absSrc), getPathName(absDst));
+     } catch (UnresolvedLinkException e) {
+       // Fully resolve the source
+       final Path source = getFileLinkStatus(absSrc).getPath();
+       // Keep trying to resolve the destination
+       return new FileSystemLinkResolver<Boolean>() {
+         @Override
+         public Boolean doCall(final Path p)
+             throws IOException, UnresolvedLinkException {
+           return dfs.rename(getPathName(source), getPathName(p));
+         }
+         @Override
+         public Boolean next(final FileSystem fs, final Path p)
+             throws IOException {
+           // Should just throw an error in FileSystem#checkPath
+           return doCall(p);
+         }
+       }.resolve(this, absDst);
+     }
+   }
+ 
+   /** 
+    * This rename operation is guaranteed to be atomic.
+    */
+   @SuppressWarnings("deprecation")
+   @Override
+   public void rename(Path src, Path dst, final Options.Rename... options)
+       throws IOException {
+     statistics.incrementWriteOps(1);
+     final Path absSrc = fixRelativePart(src);
+     final Path absDst = fixRelativePart(dst);
+     // Try the rename without resolving first
+     try {
+       dfs.rename(getPathName(absSrc), getPathName(absDst), options);
+     } catch (UnresolvedLinkException e) {
+       // Fully resolve the source
+       final Path source = getFileLinkStatus(absSrc).getPath();
+       // Keep trying to resolve the destination
+       new FileSystemLinkResolver<Void>() {
+         @Override
+         public Void doCall(final Path p)
+             throws IOException, UnresolvedLinkException {
+           dfs.rename(getPathName(source), getPathName(p), options);
+           return null;
+         }
+         @Override
+         public Void next(final FileSystem fs, final Path p)
+             throws IOException {
+           // Should just throw an error in FileSystem#checkPath
+           return doCall(p);
+         }
+       }.resolve(this, absDst);
+     }
+   }
+ 
+   @Override
+   public boolean truncate(Path f, final long newLength) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.truncate(getPathName(p), newLength);
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.truncate(p, newLength);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public boolean delete(Path f, final boolean recursive) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.delete(getPathName(p), recursive);
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.delete(p, recursive);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public ContentSummary getContentSummary(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<ContentSummary>() {
+       @Override
+       public ContentSummary doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getContentSummary(getPathName(p));
+       }
+       @Override
+       public ContentSummary next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getContentSummary(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /** Set a directory's quotas
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long, StorageType)
+    */
+   public void setQuota(Path src, final long namespaceQuota,
+       final long storagespaceQuota) throws IOException {
+     Path absF = fixRelativePart(src);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         // setQuota is not defined in FileSystem, so we only can resolve
+         // within this DFS
+         return doCall(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Set the per type storage quota of a directory.
+    *
+    * @param src target directory whose quota is to be modified.
+    * @param type storage type of the specific storage type quota to be modified.
+    * @param quota value of the specific storage type quota to be modified.
+    * Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type.
+    */
+   public void setQuotaByStorageType(
+     Path src, final StorageType type, final long quota)
+     throws IOException {
+     Path absF = fixRelativePart(src);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+         throws IOException, UnresolvedLinkException {
+         dfs.setQuotaByStorageType(getPathName(p), type, quota);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+         throws IOException {
+         // setQuotaByStorageType is not defined in FileSystem, so we only can resolve
+         // within this DFS
+         return doCall(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   private FileStatus[] listStatusInternal(Path p) throws IOException {
+     String src = getPathName(p);
+ 
+     // fetch the first batch of entries in the directory
+     DirectoryListing thisListing = dfs.listPaths(
+         src, HdfsFileStatus.EMPTY_NAME);
+ 
+     if (thisListing == null) { // the directory does not exist
+       throw new FileNotFoundException("File " + p + " does not exist.");
+     }
+     
+     HdfsFileStatus[] partialListing = thisListing.getPartialListing();
+     if (!thisListing.hasMore()) { // got all entries of the directory
+       FileStatus[] stats = new FileStatus[partialListing.length];
+       for (int i = 0; i < partialListing.length; i++) {
+         stats[i] = partialListing[i].makeQualified(getUri(), p);
+       }
+       statistics.incrementReadOps(1);
+       return stats;
+     }
+ 
+     // The directory size is too big that it needs to fetch more
+     // estimate the total number of entries in the directory
+     int totalNumEntries =
+       partialListing.length + thisListing.getRemainingEntries();
+     ArrayList<FileStatus> listing =
+       new ArrayList<FileStatus>(totalNumEntries);
+     // add the first batch of entries to the array list
+     for (HdfsFileStatus fileStatus : partialListing) {
+       listing.add(fileStatus.makeQualified(getUri(), p));
+     }
+     statistics.incrementLargeReadOps(1);
+  
+     // now fetch more entries
+     do {
+       thisListing = dfs.listPaths(src, thisListing.getLastName());
+  
+       if (thisListing == null) { // the directory is deleted
+         throw new FileNotFoundException("File " + p + " does not exist.");
+       }
+  
+       partialListing = thisListing.getPartialListing();
+       for (HdfsFileStatus fileStatus : partialListing) {
+         listing.add(fileStatus.makeQualified(getUri(), p));
+       }
+       statistics.incrementLargeReadOps(1);
+     } while (thisListing.hasMore());
+  
+     return listing.toArray(new FileStatus[listing.size()]);
+   }
+ 
+   /**
+    * List all the entries of a directory
+    *
+    * Note that this operation is not atomic for a large directory.
+    * The entries of a directory may be fetched from NameNode multiple times.
+    * It only guarantees that  each name occurs once if a directory
+    * undergoes changes between the calls.
+    */
+   @Override
+   public FileStatus[] listStatus(Path p) throws IOException {
+     Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<FileStatus[]>() {
+       @Override
+       public FileStatus[] doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return listStatusInternal(p);
+       }
+       @Override
+       public FileStatus[] next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.listStatus(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
+       final PathFilter filter)
+   throws IOException {
+     Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() {
+       @Override
+       public RemoteIterator<LocatedFileStatus> doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return new DirListingIterator<LocatedFileStatus>(p, filter, true);
+       }
+ 
+       @Override
+       public RemoteIterator<LocatedFileStatus> next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           return ((DistributedFileSystem)fs).listLocatedStatus(p, filter);
+         }
+         // symlink resolution for this methos does not work cross file systems
+         // because it is a protected method.
+         throw new IOException("Link resolution does not work with multiple " +
+             "file systems for listLocatedStatus(): " + p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+ 
+   /**
+    * Returns a remote iterator so that followup calls are made on demand
+    * while consuming the entries. This reduces memory consumption during
+    * listing of a large directory.
+    *
+    * @param p target path
+    * @return remote iterator
+    */
+   @Override
+   public RemoteIterator<FileStatus> listStatusIterator(final Path p)
+   throws IOException {
+     Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() {
+       @Override
+       public RemoteIterator<FileStatus> doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return new DirListingIterator<FileStatus>(p, false);
+       }
+ 
+       @Override
+       public RemoteIterator<FileStatus> next(final FileSystem fs, final Path p)
+           throws IOException {
+           return ((DistributedFileSystem)fs).listStatusIterator(p);
+       }
+     }.resolve(this, absF);
+ 
+   }
+ 
+   /**
+    * This class defines an iterator that returns
+    * the file status of each file/subdirectory of a directory
+    * 
+    * if needLocation, status contains block location if it is a file
+    * throws a RuntimeException with the error as its cause.
+    * 
+    * @param <T> the type of the file status
+    */
+   private class  DirListingIterator<T extends FileStatus>
+   implements RemoteIterator<T> {
+     private DirectoryListing thisListing;
+     private int i;
+     private Path p;
+     private String src;
+     private T curStat = null;
+     private PathFilter filter;
+     private boolean needLocation;
+ 
+     private DirListingIterator(Path p, PathFilter filter,
+         boolean needLocation) throws IOException {
+       this.p = p;
+       this.src = getPathName(p);
+       this.filter = filter;
+       this.needLocation = needLocation;
+       // fetch the first batch of entries in the directory
+       thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME,
+           needLocation);
+       statistics.incrementReadOps(1);
+       if (thisListing == null) { // the directory does not exist
+         throw new FileNotFoundException("File " + p + " does not exist.");
+       }
+       i = 0;
+     }
+ 
+     private DirListingIterator(Path p, boolean needLocation)
+         throws IOException {
+       this(p, null, needLocation);
+     }
+ 
+     @Override
+     @SuppressWarnings("unchecked")
+     public boolean hasNext() throws IOException {
+       while (curStat == null && hasNextNoFilter()) {
+         T next;
+         HdfsFileStatus fileStat = thisListing.getPartialListing()[i++];
+         if (needLocation) {
+           next = (T)((HdfsLocatedFileStatus)fileStat)
+               .makeQualifiedLocated(getUri(), p);
+         } else {
+           next = (T)fileStat.makeQualified(getUri(), p);
+         }
+           // apply filter if not null
+         if (filter == null || filter.accept(next.getPath())) {
+           curStat = next;
+         }
+       }
+       return curStat != null;
+     }
+       
+     /** Check if there is a next item before applying the given filter */
+     private boolean hasNextNoFilter() throws IOException {
+       if (thisListing == null) {
+         return false;
+       }
+       if (i >= thisListing.getPartialListing().length
+           && thisListing.hasMore()) { 
+         // current listing is exhausted & fetch a new listing
+         thisListing = dfs.listPaths(src, thisListing.getLastName(),
+             needLocation);
+         statistics.incrementReadOps(1);
+         if (thisListing == null) {
+           return false;
+         }
+         i = 0;
+       }
+       return (i < thisListing.getPartialListing().length);
+     }
+ 
+     @Override
+     public T next() throws IOException {
+       if (hasNext()) {
+         T tmp = curStat;
+         curStat = null;
+         return tmp;
+       } 
+       throw new java.util.NoSuchElementException("No more entry in " + p);
+     }
+   }
+   
+   /**
+    * Create a directory, only when the parent directories exist.
+    *
+    * See {@link FsPermission#applyUMask(FsPermission)} for details of how
+    * the permission is applied.
+    *
+    * @param f           The path to create
+    * @param permission  The permission.  See FsPermission#applyUMask for 
+    *                    details about how this is used to calculate the
+    *                    effective permission.
+    */
+   public boolean mkdir(Path f, FsPermission permission) throws IOException {
+     return mkdirsInternal(f, permission, false);
+   }
+ 
+   /**
+    * Create a directory and its parent directories.
+    *
+    * See {@link FsPermission#applyUMask(FsPermission)} for details of how
+    * the permission is applied.
+    *
+    * @param f           The path to create
+    * @param permission  The permission.  See FsPermission#applyUMask for 
+    *                    details about how this is used to calculate the
+    *                    effective permission.
+    */
+   @Override
+   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+     return mkdirsInternal(f, permission, true);
+   }
+ 
+   private boolean mkdirsInternal(Path f, final FsPermission permission,
+       final boolean createParent) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.mkdirs(getPathName(p), permission, createParent);
+       }
+ 
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         // FileSystem doesn't have a non-recursive mkdir() method
+         // Best we can do is error out
+         if (!createParent) {
+           throw new IOException("FileSystem does not support non-recursive"
+               + "mkdir");
+         }
+         return fs.mkdirs(p, permission);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @SuppressWarnings("deprecation")
+   @Override
+   protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
+     throws IOException {
+     statistics.incrementWriteOps(1);
+     return dfs.primitiveMkdir(getPathName(f), absolutePermission);
+   }
+ 
+  
+   @Override
+   public void close() throws IOException {
+     try {
+       dfs.closeOutputStreams(false);
+       super.close();
+     } finally {
+       dfs.close();
+     }
+   }
+ 
+   @Override
+   public String toString() {
+     return "DFS[" + dfs + "]";
+   }
+ 
+   @InterfaceAudience.Private
+   @VisibleForTesting
+   public DFSClient getClient() {
+     return dfs;
+   }        
+   
+   @Override
+   public FsStatus getStatus(Path p) throws IOException {
+     statistics.incrementReadOps(1);
+     return dfs.getDiskStatus();
+   }
+ 
+   /**
+    * Returns count of blocks with no good replicas left. Normally should be
+    * zero.
+    * 
+    * @throws IOException
+    */
+   public long getMissingBlocksCount() throws IOException {
+     return dfs.getMissingBlocksCount();
+   }
+ 
+   /**
+    * Returns count of blocks with replication factor 1 and have
+    * lost the only replica.
+    *
+    * @throws IOException
+    */
+   public long getMissingReplOneBlocksCount() throws IOException {
+     return dfs.getMissingReplOneBlocksCount();
+   }
+ 
+   /**
+    * Returns count of blocks with one of more replica missing.
+    * 
+    * @throws IOException
+    */
+   public long getUnderReplicatedBlocksCount() throws IOException {
+     return dfs.getUnderReplicatedBlocksCount();
+   }
+ 
+   /**
+    * Returns count of blocks with at least one replica marked corrupt.
+    * 
+    * @throws IOException
+    */
+   public long getCorruptBlocksCount() throws IOException {
+     return dfs.getCorruptBlocksCount();
+   }
+ 
+   @Override
+   public RemoteIterator<Path> listCorruptFileBlocks(Path path)
+     throws IOException {
+     return new CorruptFileBlockIterator(dfs, path);
+   }
+ 
+   /** @return datanode statistics. */
+   public DatanodeInfo[] getDataNodeStats() throws IOException {
+     return getDataNodeStats(DatanodeReportType.ALL);
+   }
+ 
+   /** @return datanode statistics for the given type. */
+   public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type
+       ) throws IOException {
+     return dfs.datanodeReport(type);
+   }
+ 
+   /**
+    * Enter, leave or get safe mode.
+    *  
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
+    *    HdfsConstants.SafeModeAction,boolean)
+    */
+   public boolean setSafeMode(HdfsConstants.SafeModeAction action) 
+   throws IOException {
+     return setSafeMode(action, false);
+   }
+ 
+   /**
+    * Enter, leave or get safe mode.
+    * 
+    * @param action
+    *          One of SafeModeAction.ENTER, SafeModeAction.LEAVE and
+    *          SafeModeAction.GET
+    * @param isChecked
+    *          If true check only for Active NNs status, else check first NN's
+    *          status
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(SafeModeAction, boolean)
+    */
+   public boolean setSafeMode(HdfsConstants.SafeModeAction action,
+       boolean isChecked) throws IOException {
+     return dfs.setSafeMode(action, isChecked);
+   }
+ 
+   /**
+    * Save namespace image.
+    *
+    * @param timeWindow NameNode can ignore this command if the latest
+    *                   checkpoint was done within the given time period (in
+    *                   seconds).
+    * @return true if a new checkpoint has been made
+    * @see ClientProtocol#saveNamespace(long, long)
+    */
+   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+     return dfs.saveNamespace(timeWindow, txGap);
+   }
+ 
+   /**
+    * Save namespace image. NameNode always does the checkpoint.
+    */
+   public void saveNamespace() throws IOException {
+     saveNamespace(0, 0);
+   }
+ 
+   /**
+    * Rolls the edit log on the active NameNode.
+    * Requires super-user privileges.
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits()
+    * @return the transaction ID of the newly created segment
+    */
+   public long rollEdits() throws AccessControlException, IOException {
+     return dfs.rollEdits();
+   }
+ 
+   /**
+    * enable/disable/check restoreFaileStorage
+    * 
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
+    */
+   public boolean restoreFailedStorage(String arg)
+       throws AccessControlException, IOException {
+     return dfs.restoreFailedStorage(arg);
+   }
+   
+ 
+   /**
+    * Refreshes the list of hosts and excluded hosts from the configured 
+    * files.  
+    */
+   public void refreshNodes() throws IOException {
+     dfs.refreshNodes();
+   }
+ 
+   /**
+    * Finalize previously upgraded files system state.
+    * @throws IOException
+    */
+   public void finalizeUpgrade() throws IOException {
+     dfs.finalizeUpgrade();
+   }
+ 
+   /**
+    * Rolling upgrade: prepare/finalize/query.
+    */
+   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+       throws IOException {
+     return dfs.rollingUpgrade(action);
+   }
+ 
+   /*
+    * Requests the namenode to dump data strcutures into specified 
+    * file.
+    */
+   public void metaSave(String pathname) throws IOException {
+     dfs.metaSave(pathname);
+   }
+ 
+   @Override
+   public FsServerDefaults getServerDefaults() throws IOException {
+     return dfs.getServerDefaults();
+   }
+ 
+   /**
+    * Returns the stat information about the file.
+    * @throws FileNotFoundException if the file does not exist.
+    */
+   @Override
+   public FileStatus getFileStatus(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FileStatus>() {
+       @Override
+       public FileStatus doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
+         if (fi != null) {
+           return fi.makeQualified(getUri(), p);
+         } else {
+           throw new FileNotFoundException("File does not exist: " + p);
+         }
+       }
+       @Override
+       public FileStatus next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getFileStatus(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @SuppressWarnings("deprecation")
+   @Override
+   public void createSymlink(final Path target, final Path link,
+       final boolean createParent) throws AccessControlException,
+       FileAlreadyExistsException, FileNotFoundException,
+       ParentNotDirectoryException, UnsupportedFileSystemException, 
+       IOException {
+     if (!FileSystem.areSymlinksEnabled()) {
+       throw new UnsupportedOperationException("Symlinks not supported");
+     }
+     statistics.incrementWriteOps(1);
+     final Path absF = fixRelativePart(link);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         dfs.createSymlink(target.toString(), getPathName(p), createParent);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException, UnresolvedLinkException {
+         fs.createSymlink(target, p, createParent);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public boolean supportsSymlinks() {
+     return true;
+   }
+ 
+   @Override
+   public FileStatus getFileLinkStatus(final Path f)
+       throws AccessControlException, FileNotFoundException,
+       UnsupportedFileSystemException, IOException {
+     statistics.incrementReadOps(1);
+     final Path absF = fixRelativePart(f);
+     FileStatus status = new FileSystemLinkResolver<FileStatus>() {
+       @Override
+       public FileStatus doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
+         if (fi != null) {
+           return fi.makeQualified(getUri(), p);
+         } else {
+           throw new FileNotFoundException("File does not exist: " + p);
+         }
+       }
+       @Override
+       public FileStatus next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getFileLinkStatus(p);
+       }
+     }.resolve(this, absF);
+     // Fully-qualify the symlink
+     if (status.isSymlink()) {
+       Path targetQual = FSLinkResolver.qualifySymlinkTarget(this.getUri(),
+           status.getPath(), status.getSymlink());
+       status.setSymlink(targetQual);
+     }
+     return status;
+   }
+ 
+   @Override
+   public Path getLinkTarget(final Path f) throws AccessControlException,
+       FileNotFoundException, UnsupportedFileSystemException, IOException {
+     statistics.incrementReadOps(1);
+     final Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Path>() {
+       @Override
+       public Path doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
+         if (fi != null) {
+           return fi.makeQualified(getUri(), p).getSymlink();
+         } else {
+           throw new FileNotFoundException("File does not exist: " + p);
+         }
+       }
+       @Override
+       public Path next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getLinkTarget(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   protected Path resolveLink(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     String target = dfs.getLinkTarget(getPathName(fixRelativePart(f)));
+     if (target == null) {
+       throw new FileNotFoundException("File does not exist: " + f.toString());
+     }
+     return new Path(target);
+   }
+ 
+   @Override
+   public FileChecksum getFileChecksum(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FileChecksum>() {
+       @Override
+       public FileChecksum doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
+       }
+ 
+       @Override
+       public FileChecksum next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getFileChecksum(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FileChecksum getFileChecksum(Path f, final long length)
+       throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FileChecksum>() {
+       @Override
+       public FileChecksum doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getFileChecksum(getPathName(p), length);
+       }
+ 
+       @Override
+       public FileChecksum next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           return ((DistributedFileSystem) fs).getFileChecksum(p, length);
+         } else {
+           throw new UnsupportedFileSystemException(
+               "getFileChecksum(Path, long) is not supported by "
+                   + fs.getClass().getSimpleName()); 
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setPermission(Path p, final FsPermission permission
+       ) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(p);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setPermission(getPathName(p), permission);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setPermission(p, permission);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setOwner(Path p, final String username, final String groupname
+       ) throws IOException {
+     if (username == null && groupname == null) {
+       throw new IOException("username == null && groupname == null");
+     }
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(p);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setOwner(getPathName(p), username, groupname);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setOwner(p, username, groupname);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setTimes(Path p, final long mtime, final long atime
+       ) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(p);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setTimes(getPathName(p), mtime, atime);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setTimes(p, mtime, atime);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+ 
+   @Override
+   protected int getDefaultPort() {
+     return HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
+   }
+ 
+   @Override
+   public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
+       throws IOException {
+     Token<DelegationTokenIdentifier> result =
+       dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
+     return result;
+   }
+ 
+   /**
+    * Requests the namenode to tell all datanodes to use a new, non-persistent
+    * bandwidth value for dfs.balance.bandwidthPerSec.
+    * The bandwidth parameter is the max number of bytes per second of network
+    * bandwidth to be used by a datanode during balancing.
+    *
+    * @param bandwidth Balancer bandwidth in bytes per second for all datanodes.
+    * @throws IOException
+    */
+   public void setBalancerBandwidth(long bandwidth) throws IOException {
+     dfs.setBalancerBandwidth(bandwidth);
+   }
+ 
+   /**
+    * Get a canonical service name for this file system. If the URI is logical,
+    * the hostname part of the URI will be returned.
+    * @return a service string that uniquely identifies this file system.
+    */
+   @Override
+   public String getCanonicalServiceName() {
+     return dfs.getCanonicalServiceName();
+   }
+   
+   @Override
+   protected URI canonicalizeUri(URI uri) {
+     if (HAUtilClient.isLogicalUri(getConf(), uri)) {
+       // Don't try to DNS-resolve logical URIs, since the 'authority'
+       // portion isn't a proper hostname
+       return uri;
+     } else {
+       return NetUtils.getCanonicalUri(uri, getDefaultPort());
+     }
+   }
+ 
+   /**
+    * Utility function that returns if the NameNode is in safemode or not. In HA
+    * mode, this API will return only ActiveNN's safemode status.
+    * 
+    * @return true if NameNode is in safemode, false otherwise.
+    * @throws IOException
+    *           when there is an issue communicating with the NameNode
+    */
+   public boolean isInSafeMode() throws IOException {
+     return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
+   }
+ 
+   /** @see HdfsAdmin#allowSnapshot(Path) */
+   public void allowSnapshot(final Path path) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.allowSnapshot(getPathName(p));
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.allowSnapshot(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+   /** @see HdfsAdmin#disallowSnapshot(Path) */
+   public void disallowSnapshot(final Path path) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.disallowSnapshot(getPathName(p));
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.disallowSnapshot(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public Path createSnapshot(final Path path, final String snapshotName) 
+       throws IOException {
+     Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<Path>() {
+       @Override
+       public Path doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return new Path(dfs.createSnapshot(getPathName(p), snapshotName));
+       }
+ 
+       @Override
+       public Path next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.createSnapshot(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public void renameSnapshot(final Path path, final String snapshotOldName,
+       final String snapshotNewName) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.renameSnapshot(p, snapshotOldName, snapshotNewName);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+   /**
+    * @return All the snapshottable directories
+    * @throws IOException
+    */
+   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+       throws IOException {
+     return dfs.getSnapshottableDirListing();
+   }
+   
+   @Override
+   public void deleteSnapshot(final Path snapshotDir, final String snapshotName)
+       throws IOException {
+     Path absF = fixRelativePart(snapshotDir);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.deleteSnapshot(getPathName(p), snapshotName);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.deleteSnapshot(p, snapshotName);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + snapshotDir + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Get the difference between two snapshots, or between a snapshot and the
+    * current tree of a directory.
+    * 
+    * @see DFSClient#getSnapshotDiffReport(String, String, String)
+    */
+   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
+       final String fromSnapshot, final String toSnapshot) throws IOException {
+     Path absF = fixRelativePart(snapshotDir);
+     return new FileSystemLinkResolver<SnapshotDiffReport>() {
+       @Override
+       public SnapshotDiffReport doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
+             toSnapshot);
+       }
+ 
+       @Override
+       public SnapshotDiffReport next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.getSnapshotDiffReport(p, fromSnapshot, toSnapshot);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + snapshotDir + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+  
+   /**
+    * Get the close status of a file
+    * @param src The path to the file
+    *
+    * @return return true if file is closed
+    * @throws FileNotFoundException if the file does not exist.
+    * @throws IOException If an I/O error occurred     
+    */
+   public boolean isFileClosed(final Path src) throws IOException {
+     Path absF = fixRelativePart(src);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.isFileClosed(getPathName(p));
+       }
+ 
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.isFileClosed(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot call isFileClosed"
+               + " on a symlink to a non-DistributedFileSystem: "
+               + src + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * @see {@link #addCacheDirective(CacheDirectiveInfo, EnumSet)}
+    */
+   public long addCacheDirective(CacheDirectiveInfo info) throws IOException {
+     return addCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+   }
+ 
+   /**
+    * Add a new CacheDirective.
+    * 
+    * @param info Information about a directive to add.
+    * @param flags {@link CacheFlag}s to use for this operation.
+    * @return the ID of the directive that was created.
+    * @throws IOException if the directive could not be added
+    */
+   public long addCacheDirective(
+       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+     Preconditions.checkNotNull(info.getPath());
+     Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
+         makeQualified(getUri(), getWorkingDirectory());
+     return dfs.addCacheDirective(
+         new CacheDirectiveInfo.Builder(info).
+             setPath(path).
+             build(),
+         flags);
+   }
+ 
+   /**
+    * @see {@link #modifyCacheDirective(CacheDirectiveInfo, EnumSet)}
+    */
+   public void modifyCacheDirective(CacheDirectiveInfo info) throws IOException {
+     modifyCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+   }
+ 
+   /**
+    * Modify a CacheDirective.
+    * 
+    * @param info Information about the directive to modify. You must set the ID
+    *          to indicate which CacheDirective you want to modify.
+    * @param flags {@link CacheFlag}s to use for this operation.
+    * @throws IOException if the directive could not be modified
+    */
+   public void modifyCacheDirective(
+       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+     if (info.getPath() != null) {
+       info = new CacheDirectiveInfo.Builder(info).
+           setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
+               makeQualified(getUri(), getWorkingDirectory())).build();
+     }
+     dfs.modifyCacheDirective(info, flags);
+   }
+ 
+   /**
+    * Remove a CacheDirectiveInfo.
+    * 
+    * @param id identifier of the CacheDirectiveInfo to remove
+    * @throws IOException if the directive could not be removed
+    */
+   public void removeCacheDirective(long id)
+       throws IOException {
+     dfs.removeCacheDirective(id);
+   }
+   
+   /**
+    * List cache directives.  Incrementally fetches results from the server.
+    * 
+    * @param filter Filter parameters to use when listing the directives, null to
+    *               list all directives visible to us.
+    * @return A RemoteIterator which returns CacheDirectiveInfo objects.
+    */
+   public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+       CacheDirectiveInfo filter) throws IOException {
+     if (filter == null) {
+       filter = new CacheDirectiveInfo.Builder().build();
+     }
+     if (filter.getPath() != null) {
+       filter = new CacheDirectiveInfo.Builder(filter).
+           setPath(new Path(getPathName(fixRelativePart(filter.getPath())))).
+           build();
+     }
+     final RemoteIterator<CacheDirectiveEntry> iter =
+         dfs.listCacheDirectives(filter);
+     return new RemoteIterator<CacheDirectiveEntry>() {
+       @Override
+       public boolean hasNext() throws IOException {
+         return iter.hasNext();
+       }
+ 
+       @Override
+       public CacheDirectiveEntry next() throws IOException {
+         // Although the paths we get back from the NameNode should always be
+         // absolute, we call makeQualified to add the scheme and authority of
+         // this DistributedFilesystem.
+         CacheDirectiveEntry desc = iter.next();
+         CacheDirectiveInfo info = desc.getInfo();
+         Path p = info.getPath().makeQualified(getUri(), getWorkingDirectory());
+         return new CacheDirectiveEntry(
+             new CacheDirectiveInfo.Builder(info).setPath(p).build(),
+             desc.getStats());
+       }
+     };
+   }
+ 
+   /**
+    * Add a cache pool.
+    *
+    * @param info
+    *          The request to add a cache pool.
+    * @throws IOException 
+    *          If the request could not be completed.
+    */
+   public void addCachePool(CachePoolInfo info) throws IOException {
+     CachePoolInfo.validate(info);
+     dfs.addCachePool(info);
+   }
+ 
+   /**
+    * Modify an existing cache pool.
+    *
+    * @param info
+    *          The request to modify a cache pool.
+    * @throws IOException 
+    *          If the request could not be completed.
+    */
+   public void modifyCachePool(CachePoolInfo info) throws IOException {
+     CachePoolInfo.validate(info);
+     dfs.modifyCachePool(info);
+   }
+     
+   /**
+    * Remove a cache pool.
+    *
+    * @param poolName
+    *          Name of the cache pool to remove.
+    * @throws IOException 
+    *          if the cache pool did not exist, or could not be removed.
+    */
+   public void removeCachePool(String poolName) throws IOException {
+     CachePoolInfo.validateName(poolName);
+     dfs.removeCachePool(poolName);
+   }
+ 
+   /**
+    * List all cache pools.
+    *
+    * @return A remote iterator from which you can get CachePoolEntry objects.
+    *          Requests will be made as needed.
+    * @throws IOException
+    *          If there was an error listing cache pools.
+    */
+   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+     return dfs.listCachePools();
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void modifyAclEntries(Path path, final List<AclEntry> aclSpec)
+       throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.modifyAclEntries(getPathName(p), aclSpec);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.modifyAclEntries(p, aclSpec);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void removeAclEntries(Path path, final List<AclEntry> aclSpec)
+       throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeAclEntries(getPathName(p), aclSpec);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.removeAclEntries(p, aclSpec);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void removeDefaultAcl(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeDefaultAcl(getPathName(p));
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         fs.removeDefaultAcl(p);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void removeAcl(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeAcl(getPathName(p));
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         fs.removeAcl(p);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void setAcl(Path path, final List<AclEntry> aclSpec) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.setAcl(getPathName(p), aclSpec);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.setAcl(p, aclSpec);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public AclStatus getAclStatus(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<AclStatus>() {
+       @Override
+       public AclStatus doCall(final Path p) throws IOException {
+         return dfs.getAclStatus(getPathName(p));
+       }
+       @Override
+       public AclStatus next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getAclStatus(p);
+       }
+     }.resolve(this, absF);
+   }
+   
+   /* HDFS only */
+   public void createEncryptionZone(final Path path, final String keyName)
+     throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         dfs.createEncryptionZone(getPathName(p), keyName);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+           myDfs.createEncryptionZone(p, keyName);
+           return null;
+         } else {
+           throw new UnsupportedOperationException(
+               "Cannot call createEncryptionZone"
+                   + " on a symlink to a non-DistributedFileSystem: " + path
+                   + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /* HDFS only */
+   public EncryptionZone getEZForPath(final Path path)
+           throws IOException {
+     Preconditions.checkNotNull(path);
+     Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<EncryptionZone>() {
+       @Override
+       public EncryptionZone doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         return dfs.getEZForPath(getPathName(p));
+       }
+ 
+       @Override
+       public EncryptionZone next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+           return myDfs.getEZForPath(p);
+         } else {
+           throw new UnsupportedOperationException(
+               "Cannot call getEZForPath"
+                   + " on a symlink to a non-DistributedFileSystem: " + path
+                   + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /* HDFS only */
+   public RemoteIterator<EncryptionZone> listEncryptionZones()
+       throws IOException {
+     return dfs.listEncryptionZones();
+   }
+ 
+   @Override
+   public void setXAttr(Path path, final String name, final byte[] value, 
+       final EnumSet<XAttrSetFlag> flag) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+ 
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.setXAttr(getPathName(p), name, value, flag);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.setXAttr(p, name, value, flag);
+         return null;
+       }      
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public byte[] getXAttr(Path path, final String name) throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<byte[]>() {
+       @Override
+       public byte[] doCall(final Path p) throws IOException {
+         return dfs.getXAttr(getPathName(p), name);
+       }
+       @Override
+       public byte[] next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getXAttr(p, name);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public Map<String, byte[]> getXAttrs(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<Map<String, byte[]>>() {
+       @Override
+       public Map<String, byte[]> doCall(final Path p) throws IOException {
+         return dfs.getXAttrs(getPathName(p));
+       }
+       @Override
+       public Map<String, byte[]> next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getXAttrs(p);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public Map<String, byte[]> getXAttrs(Path path, final List<String> names) 
+       throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<Map<String, byte[]>>() {
+       @Override
+       public Map<String, byte[]> doCall(final Path p) throws IOException {
+         return dfs.getXAttrs(getPathName(p), names);
+       }
+       @Override
+       public Map<String, byte[]> next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getXAttrs(p, names);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public List<String> listXAttrs(Path path)
+           throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<List<String>>() {
+       @Override
+       public List<String> doCall(final Path p) throws IOException {
+         return dfs.listXAttrs(getPathName(p));
+       }
+       @Override
+       public List<String> next(final FileSystem fs, final Path p)
+               throws IOException, UnresolvedLinkException {
+         return fs.listXAttrs(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void removeXAttr(Path path, final String name) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeXAttr(getPathName(p), name);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.removeXAttr(p, name);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void access(Path path, final FsAction mode) throws IOException {
+     final Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.checkAccess(getPathName(p), mode);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.access(p, mode);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public Token<?>[] addDelegationTokens(
+       final String renewer, Credentials credentials) throws IOException {
+     Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
+     if (dfs.isHDFSEncryptionEnabled()) {
+       KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension =
+           KeyProviderDelegationTokenExtension.
+               createKeyProviderDelegationTokenExtension(dfs.getKeyProvider());
+       Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
+           addDelegationTokens(renewer, credentials);
+       if (tokens != null && kpTokens != null) {
+         Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
+         System.arraycopy(tokens, 0, all, 0, tokens.length);
+         System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
+         tokens = all;
+       } else {
+         tokens = (tokens != null) ? tokens : kpTokens;
+       }
+     }
+     return tokens;
+   }
+ 
+   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
+     return dfs.getInotifyEventStream();
+   }
+ 
+   public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
+       throws IOException {
+     return dfs.getInotifyEventStream(lastReadTxid);
+   }
++
++  /**
++   * Set the source path to the specified erasure coding policy.
++   *
++   * @param path     The directory to set the policy
++   * @param ecPolicy The erasure coding policy. If not specified default will be used.
++   * @throws IOException
++   */
++  public void setErasureCodingPolicy(final Path path, final ErasureCodingPolicy ecPolicy)
++      throws IOException {
++    Path absF = fixRelativePart(path);
++    new FileSystemLinkResolver<Void>() {
++      @Override
++      public Void doCall(final Path p) throws IOException,
++          UnresolvedLinkException {
++        dfs.setErasureCodingPolicy(getPathName(p), ecPolicy);
++        return null;
++      }
++
++      @Override
++      public Void next(final FileSystem fs, final Path p) throws IOException {
++        if (fs instanceof DistributedFileSystem) {
++          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
++          myDfs.setErasureCodingPolicy(p, ecPolicy);
++          return null;
++        }
++        throw new UnsupportedOperationException(
++            "Cannot setErasureCodingPolicy through a symlink to a "
++                + "non-DistributedFileSystem: " + path + " -> " + p);
++      }
++    }.resolve(this, absF);
++  }
++
++  /**
++   * Get erasure coding policy information for the specified path
++   *
++   * @param path The path of the file or directory
++   * @return Returns the policy information if file or directory on the path
++   * is erasure coded, null otherwise
++   * @throws IOException
++   */
++  public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
++      throws IOException {
++    Path absF = fixRelativePart(path);
++    return new FileSystemLinkResolver<ErasureCodingPolicy>() {
++      @Override
++      public ErasureCodingPolicy doCall(final Path p) throws IOException,
++          UnresolvedLinkException {
++        return dfs.getErasureCodingPolicy(getPathName(p));
++      }
++
++      @Override
++      public ErasureCodingPolicy next(final FileSystem fs, final Path p)
++          throws IOException {
++        if (fs instanceof DistributedFileSystem) {
++          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
++          return myDfs.getErasureCodingPolicy(p);
++        }
++        throw new UnsupportedOperationException(
++            "Cannot getErasureCodingPolicy through a symlink to a "
++                + "non-DistributedFileSystem: " + path + " -> " + p);
++      }
++    }.resolve(this, absF);
++  }
++
++  /**
++   * Retrieve all the erasure coding policies supported by this file system.
++   *
++   * @return all erasure coding policies supported by this file system.
++   * @throws IOException
++   */
++  public Collection<ErasureCodingPolicy> getAllErasureCodingPolicies()
++      throws IOException {
++    return Arrays.asList(dfs.getErasureCodingPolicies());
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------


[19/58] [abbrv] hadoop git commit: HDFS-9080. Update htrace version to 4.0.1 (cmccabe)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index 67cd524..92d117c 100644
--- 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
@@ -74,6 +74,7 @@ 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.FsTracer;
 import org.apache.hadoop.fs.HdfsBlockLocation;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
@@ -166,24 +167,19 @@ 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 org.apache.htrace.core.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.apache.htrace.core.Tracer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -206,6 +202,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
 
   private final Configuration conf;
+  private final Tracer tracer;
   private final DfsClientConf dfsClientConf;
   final ClientProtocol namenode;
   /* The service used for delegation tokens */
@@ -232,7 +229,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   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() {
@@ -296,11 +292,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   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.tracer = FsTracer.get(conf);
     this.dfsClientConf = new DfsClientConf(conf);
     this.conf = conf;
     this.stats = stats;
@@ -623,7 +616,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public long getBlockSize(String f) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getBlockSize", f);
+    TraceScope scope = newPathTraceScope("getBlockSize", f);
     try {
       return namenode.getPreferredBlockSize(f);
     } catch (IOException ie) {
@@ -666,7 +659,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
     assert dtService != null;
-    TraceScope scope = Trace.startSpan("getDelegationToken", traceSampler);
+    TraceScope scope = tracer.newScope("getDelegationToken");
     try {
       Token<DelegationTokenIdentifier> token =
         namenode.getDelegationToken(renewer);
@@ -823,7 +816,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   @VisibleForTesting
   public LocatedBlocks getLocatedBlocks(String src, long start, long length)
       throws IOException {
-    TraceScope scope = getPathTraceScope("getBlockLocations", src);
+    TraceScope scope = newPathTraceScope("getBlockLocations", src);
     try {
       return callGetBlockLocations(namenode, src, start, length);
     } finally {
@@ -855,7 +848,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   boolean recoverLease(String src) throws IOException {
     checkOpen();
 
-    TraceScope scope = getPathTraceScope("recoverLease", src);
+    TraceScope scope = newPathTraceScope("recoverLease", src);
     try {
       return namenode.recoverLease(src, clientName);
     } catch (RemoteException re) {
@@ -882,7 +875,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public BlockLocation[] getBlockLocations(String src, long start, 
         long length) throws IOException, UnresolvedLinkException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getBlockLocations", src);
+    TraceScope scope = newPathTraceScope("getBlockLocations", src);
     try {
       LocatedBlocks blocks = getLocatedBlocks(src, start, length);
       BlockLocation[] locations =  DFSUtilClient.locatedBlocks2Locations(blocks);
@@ -901,7 +894,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
       feInfo) throws IOException {
-    TraceScope scope = Trace.startSpan("decryptEDEK", traceSampler);
+    TraceScope scope = tracer.newScope("decryptEDEK");
     try {
       KeyProvider provider = getKeyProvider();
       if (provider == null) {
@@ -1057,7 +1050,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throws IOException, UnresolvedLinkException {
     checkOpen();
     //    Get block info from namenode
-    TraceScope scope = getPathTraceScope("newDFSInputStream", src);
+    TraceScope scope = newPathTraceScope("newDFSInputStream", src);
     try {
       return new DFSInputStream(this, src, verifyChecksum, null);
     } finally {
@@ -1303,7 +1296,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void createSymlink(String target, String link, boolean createParent)
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("createSymlink", target);
+    TraceScope scope = newPathTraceScope("createSymlink", target);
     try {
       final FsPermission dirPerm = applyUMask(null);
       namenode.createSymlink(target, link, dirPerm, createParent);
@@ -1329,7 +1322,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public String getLinkTarget(String path) throws IOException { 
     checkOpen();
-    TraceScope scope = getPathTraceScope("getLinkTarget", path);
+    TraceScope scope = newPathTraceScope("getLinkTarget", path);
     try {
       return namenode.getLinkTarget(path);
     } catch (RemoteException re) {
@@ -1426,7 +1419,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public boolean setReplication(String src, short replication)
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("setReplication", src);
+    TraceScope scope = newPathTraceScope("setReplication", src);
     try {
       return namenode.setReplication(src, replication);
     } catch(RemoteException re) {
@@ -1450,7 +1443,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("setStoragePolicy", src);
+    TraceScope scope = newPathTraceScope("setStoragePolicy", src);
     try {
       namenode.setStoragePolicy(src, policyName);
     } catch (RemoteException e) {
@@ -1471,7 +1464,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getStoragePolicy", path);
+    TraceScope scope = newPathTraceScope("getStoragePolicy", path);
     try {
       return namenode.getStoragePolicy(path);
     } catch (RemoteException e) {
@@ -1489,7 +1482,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler);
+    TraceScope scope = tracer.newScope("getStoragePolicies");
     try {
       return namenode.getStoragePolicies();
     } finally {
@@ -1505,7 +1498,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   @Deprecated
   public boolean rename(String src, String dst) throws IOException {
     checkOpen();
-    TraceScope scope = getSrcDstTraceScope("rename", src, dst);
+    TraceScope scope = newSrcDstTraceScope("rename", src, dst);
     try {
       return namenode.rename(src, dst);
     } catch(RemoteException re) {
@@ -1526,7 +1519,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void concat(String trg, String [] srcs) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("concat", traceSampler);
+    TraceScope scope = tracer.newScope("concat");
     try {
       namenode.concat(trg, srcs);
     } catch(RemoteException re) {
@@ -1544,7 +1537,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void rename(String src, String dst, Options.Rename... options)
       throws IOException {
     checkOpen();
-    TraceScope scope = getSrcDstTraceScope("rename2", src, dst);
+    TraceScope scope = newSrcDstTraceScope("rename2", src, dst);
     try {
       namenode.rename2(src, dst, options);
     } catch(RemoteException re) {
@@ -1573,7 +1566,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throw new HadoopIllegalArgumentException(
           "Cannot truncate to a negative file size: " + newLength + ".");
     }
-    TraceScope scope = getPathTraceScope("truncate", src);
+    TraceScope scope = newPathTraceScope("truncate", src);
     try {
       return namenode.truncate(src, newLength, clientName);
     } catch (RemoteException re) {
@@ -1603,7 +1596,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean delete(String src, boolean recursive) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("delete", src);
+    TraceScope scope = newPathTraceScope("delete", src);
     try {
       return namenode.delete(src, recursive);
     } catch(RemoteException re) {
@@ -1645,7 +1638,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public DirectoryListing listPaths(String src,  byte[] startAfter,
       boolean needLocation) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("listPaths", src);
+    TraceScope scope = newPathTraceScope("listPaths", src);
     try {
       return namenode.getListing(src, startAfter, needLocation);
     } catch(RemoteException re) {
@@ -1667,7 +1660,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public HdfsFileStatus getFileInfo(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getFileInfo", src);
+    TraceScope scope = newPathTraceScope("getFileInfo", src);
     try {
       return namenode.getFileInfo(src);
     } catch(RemoteException re) {
@@ -1685,7 +1678,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean isFileClosed(String src) throws IOException{
     checkOpen();
-    TraceScope scope = getPathTraceScope("isFileClosed", src);
+    TraceScope scope = newPathTraceScope("isFileClosed", src);
     try {
       return namenode.isFileClosed(src);
     } catch(RemoteException re) {
@@ -1707,7 +1700,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getFileLinkInfo", src);
+    TraceScope scope = newPathTraceScope("getFileLinkInfo", src);
     try {
       return namenode.getFileLinkInfo(src);
     } catch(RemoteException re) {
@@ -2010,7 +2003,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void setPermission(String src, FsPermission permission)
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("setPermission", src);
+    TraceScope scope = newPathTraceScope("setPermission", src);
     try {
       namenode.setPermission(src, permission);
     } catch(RemoteException re) {
@@ -2035,7 +2028,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void setOwner(String src, String username, String groupname)
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("setOwner", src);
+    TraceScope scope = newPathTraceScope("setOwner", src);
     try {
       namenode.setOwner(src, username, groupname);
     } catch(RemoteException re) {
@@ -2051,7 +2044,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   private long[] callGetStats() throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("getStats", traceSampler);
+    TraceScope scope = tracer.newScope("getStats");
     try {
       return namenode.getStats();
     } finally {
@@ -2110,7 +2103,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                                  String cookie)
         throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("listCorruptFileBlocks", path);
+    TraceScope scope = newPathTraceScope("listCorruptFileBlocks", path);
     try {
       return namenode.listCorruptFileBlocks(path, cookie);
     } finally {
@@ -2121,7 +2114,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public DatanodeInfo[] datanodeReport(DatanodeReportType type)
       throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("datanodeReport", traceSampler);
+    TraceScope scope = tracer.newScope("datanodeReport");
     try {
       return namenode.getDatanodeReport(type);
     } finally {
@@ -2133,7 +2126,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       DatanodeReportType type) throws IOException {
     checkOpen();
     TraceScope scope =
-        Trace.startSpan("datanodeStorageReport", traceSampler);
+        tracer.newScope("datanodeStorageReport");
     try {
       return namenode.getDatanodeStorageReport(type);
     } finally {
@@ -2164,7 +2157,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
     TraceScope scope =
-        Trace.startSpan("setSafeMode", traceSampler);
+        tracer.newScope("setSafeMode");
     try {
       return namenode.setSafeMode(action, isChecked);
     } finally {
@@ -2183,7 +2176,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("createSnapshot", traceSampler);
+    TraceScope scope = tracer.newScope("createSnapshot");
     try {
       return namenode.createSnapshot(snapshotRoot, snapshotName);
     } catch(RemoteException re) {
@@ -2205,7 +2198,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("deleteSnapshot", traceSampler);
+    TraceScope scope = tracer.newScope("deleteSnapshot");
     try {
       namenode.deleteSnapshot(snapshotRoot, snapshotName);
     } catch(RemoteException re) {
@@ -2226,7 +2219,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void renameSnapshot(String snapshotDir, String snapshotOldName,
       String snapshotNewName) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("renameSnapshot", traceSampler);
+    TraceScope scope = tracer.newScope("renameSnapshot");
     try {
       namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
     } catch(RemoteException re) {
@@ -2245,8 +2238,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("getSnapshottableDirListing",
-        traceSampler);
+    TraceScope scope = tracer.newScope("getSnapshottableDirListing");
     try {
       return namenode.getSnapshottableDirListing();
     } catch(RemoteException re) {
@@ -2263,7 +2255,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void allowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("allowSnapshot", traceSampler);
+    TraceScope scope = tracer.newScope("allowSnapshot");
     try {
       namenode.allowSnapshot(snapshotRoot);
     } catch (RemoteException re) {
@@ -2280,7 +2272,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void disallowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("disallowSnapshot", traceSampler);
+    TraceScope scope = tracer.newScope("disallowSnapshot");
     try {
       namenode.disallowSnapshot(snapshotRoot);
     } catch (RemoteException re) {
@@ -2298,7 +2290,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
       String fromSnapshot, String toSnapshot) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("getSnapshotDiffReport", traceSampler);
+    TraceScope scope = tracer.newScope("getSnapshotDiffReport");
     try {
       return namenode.getSnapshotDiffReport(snapshotDir,
           fromSnapshot, toSnapshot);
@@ -2312,7 +2304,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public long addCacheDirective(
       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("addCacheDirective", traceSampler);
+    TraceScope scope = tracer.newScope("addCacheDirective");
     try {
       return namenode.addCacheDirective(info, flags);
     } catch (RemoteException re) {
@@ -2325,7 +2317,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void modifyCacheDirective(
       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("modifyCacheDirective", traceSampler);
+    TraceScope scope = tracer.newScope("modifyCacheDirective");
     try {
       namenode.modifyCacheDirective(info, flags);
     } catch (RemoteException re) {
@@ -2338,7 +2330,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void removeCacheDirective(long id)
       throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("removeCacheDirective", traceSampler);
+    TraceScope scope = tracer.newScope("removeCacheDirective");
     try {
       namenode.removeCacheDirective(id);
     } catch (RemoteException re) {
@@ -2351,12 +2343,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
       CacheDirectiveInfo filter) throws IOException {
     checkOpen();
-    return new CacheDirectiveIterator(namenode, filter, traceSampler);
+    return new CacheDirectiveIterator(namenode, filter, tracer);
   }
 
   public void addCachePool(CachePoolInfo info) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("addCachePool", traceSampler);
+    TraceScope scope = tracer.newScope("addCachePool");
     try {
       namenode.addCachePool(info);
     } catch (RemoteException re) {
@@ -2368,7 +2360,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public void modifyCachePool(CachePoolInfo info) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("modifyCachePool", traceSampler);
+    TraceScope scope = tracer.newScope("modifyCachePool");
     try {
       namenode.modifyCachePool(info);
     } catch (RemoteException re) {
@@ -2380,7 +2372,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public void removeCachePool(String poolName) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("removeCachePool", traceSampler);
+    TraceScope scope = tracer.newScope("removeCachePool");
     try {
       namenode.removeCachePool(poolName);
     } catch (RemoteException re) {
@@ -2392,7 +2384,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
     checkOpen();
-    return new CachePoolIterator(namenode, traceSampler);
+    return new CachePoolIterator(namenode, tracer);
   }
 
   /**
@@ -2402,7 +2394,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   boolean saveNamespace(long timeWindow, long txGap) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("saveNamespace", traceSampler);
+    TraceScope scope = tracer.newScope("saveNamespace");
     try {
       return namenode.saveNamespace(timeWindow, txGap);
     } catch(RemoteException re) {
@@ -2420,7 +2412,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   long rollEdits() throws AccessControlException, IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("rollEdits", traceSampler);
+    TraceScope scope = tracer.newScope("rollEdits");
     try {
       return namenode.rollEdits();
     } catch(RemoteException re) {
@@ -2443,7 +2435,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   boolean restoreFailedStorage(String arg)
       throws AccessControlException, IOException{
     checkOpen();
-    TraceScope scope = Trace.startSpan("restoreFailedStorage", traceSampler);
+    TraceScope scope = tracer.newScope("restoreFailedStorage");
     try {
       return namenode.restoreFailedStorage(arg);
     } finally {
@@ -2460,7 +2452,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void refreshNodes() throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("refreshNodes", traceSampler);
+    TraceScope scope = tracer.newScope("refreshNodes");
     try {
       namenode.refreshNodes();
     } finally {
@@ -2475,7 +2467,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void metaSave(String pathname) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("metaSave", traceSampler);
+    TraceScope scope = tracer.newScope("metaSave");
     try {
       namenode.metaSave(pathname);
     } finally {
@@ -2493,7 +2485,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void setBalancerBandwidth(long bandwidth) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("setBalancerBandwidth", traceSampler);
+    TraceScope scope = tracer.newScope("setBalancerBandwidth");
     try {
       namenode.setBalancerBandwidth(bandwidth);
     } finally {
@@ -2506,7 +2498,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void finalizeUpgrade() throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("finalizeUpgrade", traceSampler);
+    TraceScope scope = tracer.newScope("finalizeUpgrade");
     try {
       namenode.finalizeUpgrade();
     } finally {
@@ -2516,7 +2508,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("rollingUpgrade", traceSampler);
+    TraceScope scope = tracer.newScope("rollingUpgrade");
     try {
       return namenode.rollingUpgrade(action);
     } finally {
@@ -2574,7 +2566,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if(LOG.isDebugEnabled()) {
       LOG.debug(src + ": masked=" + absPermission);
     }
-    TraceScope scope = Trace.startSpan("mkdir", traceSampler);
+    TraceScope scope = tracer.newScope("mkdir");
     try {
       return namenode.mkdirs(src, absPermission, createParent);
     } catch(RemoteException re) {
@@ -2602,7 +2594,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   ContentSummary getContentSummary(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getContentSummary", src);
+    TraceScope scope = newPathTraceScope("getContentSummary", src);
     try {
       return namenode.getContentSummary(src);
     } catch(RemoteException re) {
@@ -2631,7 +2623,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                          storagespaceQuota);
                                          
     }
-    TraceScope scope = getPathTraceScope("setQuota", src);
+    TraceScope scope = newPathTraceScope("setQuota", src);
     try {
       // Pass null as storage type for traditional namespace/storagespace quota.
       namenode.setQuota(src, namespaceQuota, storagespaceQuota, null);
@@ -2667,7 +2659,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throw new IllegalArgumentException("Don't support Quota for storage type : "
         + type.toString());
     }
-    TraceScope scope = getPathTraceScope("setQuotaByStorageType", src);
+    TraceScope scope = newPathTraceScope("setQuotaByStorageType", src);
     try {
       namenode.setQuota(src, HdfsConstants.QUOTA_DONT_SET, quota, type);
     } catch (RemoteException re) {
@@ -2687,7 +2679,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void setTimes(String src, long mtime, long atime) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("setTimes", src);
+    TraceScope scope = newPathTraceScope("setTimes", src);
     try {
       namenode.setTimes(src, mtime, atime);
     } catch(RemoteException re) {
@@ -2748,7 +2740,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("modifyAclEntries", src);
+    TraceScope scope = newPathTraceScope("modifyAclEntries", src);
     try {
       namenode.modifyAclEntries(src, aclSpec);
     } catch(RemoteException re) {
@@ -2767,7 +2759,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void removeAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("removeAclEntries", traceSampler);
+    TraceScope scope = tracer.newScope("removeAclEntries");
     try {
       namenode.removeAclEntries(src, aclSpec);
     } catch(RemoteException re) {
@@ -2785,7 +2777,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public void removeDefaultAcl(String src) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("removeDefaultAcl", traceSampler);
+    TraceScope scope = tracer.newScope("removeDefaultAcl");
     try {
       namenode.removeDefaultAcl(src);
     } catch(RemoteException re) {
@@ -2803,7 +2795,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public void removeAcl(String src) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("removeAcl", traceSampler);
+    TraceScope scope = tracer.newScope("removeAcl");
     try {
       namenode.removeAcl(src);
     } catch(RemoteException re) {
@@ -2821,7 +2813,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
     checkOpen();
-    TraceScope scope = Trace.startSpan("setAcl", traceSampler);
+    TraceScope scope = tracer.newScope("setAcl");
     try {
       namenode.setAcl(src, aclSpec);
     } catch(RemoteException re) {
@@ -2839,7 +2831,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public AclStatus getAclStatus(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getAclStatus", src);
+    TraceScope scope = newPathTraceScope("getAclStatus", src);
     try {
       return namenode.getAclStatus(src);
     } catch(RemoteException re) {
@@ -2855,7 +2847,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void createEncryptionZone(String src, String keyName)
     throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("createEncryptionZone", src);
+    TraceScope scope = newPathTraceScope("createEncryptionZone", src);
     try {
       namenode.createEncryptionZone(src, keyName);
     } catch (RemoteException re) {
@@ -2870,7 +2862,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public EncryptionZone getEZForPath(String src)
           throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getEZForPath", src);
+    TraceScope scope = newPathTraceScope("getEZForPath", src);
     try {
       return namenode.getEZForPath(src);
     } catch (RemoteException re) {
@@ -2884,13 +2876,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public RemoteIterator<EncryptionZone> listEncryptionZones()
       throws IOException {
     checkOpen();
-    return new EncryptionZoneIterator(namenode, traceSampler);
+    return new EncryptionZoneIterator(namenode, tracer);
   }
 
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("setXAttr", src);
+    TraceScope scope = newPathTraceScope("setXAttr", src);
     try {
       namenode.setXAttr(src, XAttrHelper.buildXAttr(name, value), flag);
     } catch (RemoteException re) {
@@ -2907,7 +2899,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   
   public byte[] getXAttr(String src, String name) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getXAttr", src);
+    TraceScope scope = newPathTraceScope("getXAttr", src);
     try {
       final List<XAttr> xAttrs = XAttrHelper.buildXAttrAsList(name);
       final List<XAttr> result = namenode.getXAttrs(src, xAttrs);
@@ -2923,7 +2915,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   
   public Map<String, byte[]> getXAttrs(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getXAttrs", src);
+    TraceScope scope = newPathTraceScope("getXAttrs", src);
     try {
       return XAttrHelper.buildXAttrMap(namenode.getXAttrs(src, null));
     } catch(RemoteException re) {
@@ -2938,7 +2930,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public Map<String, byte[]> getXAttrs(String src, List<String> names) 
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getXAttrs", src);
+    TraceScope scope = newPathTraceScope("getXAttrs", src);
     try {
       return XAttrHelper.buildXAttrMap(namenode.getXAttrs(
           src, XAttrHelper.buildXAttrs(names)));
@@ -2954,7 +2946,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public List<String> listXAttrs(String src)
           throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("listXAttrs", src);
+    TraceScope scope = newPathTraceScope("listXAttrs", src);
     try {
       final Map<String, byte[]> xattrs =
         XAttrHelper.buildXAttrMap(namenode.listXAttrs(src));
@@ -2970,7 +2962,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public void removeXAttr(String src, String name) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("removeXAttr", src);
+    TraceScope scope = newPathTraceScope("removeXAttr", src);
     try {
       namenode.removeXAttr(src, XAttrHelper.buildXAttr(name));
     } catch(RemoteException re) {
@@ -2987,7 +2979,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public void checkAccess(String src, FsAction mode) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("checkAccess", src);
+    TraceScope scope = newPathTraceScope("checkAccess", src);
     try {
       namenode.checkAccess(src, mode);
     } catch (RemoteException re) {
@@ -3001,13 +2993,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
     checkOpen();
-    return new DFSInotifyEventInputStream(traceSampler, namenode);
+    return new DFSInotifyEventInputStream(namenode, tracer);
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
       throws IOException {
     checkOpen();
-    return new DFSInotifyEventInputStream(traceSampler, namenode, lastReadTxid);
+    return new DFSInotifyEventInputStream(namenode, tracer,
+          lastReadTxid);
   }
 
   @Override // RemotePeerFactory
@@ -3117,28 +3110,26 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return saslClient;
   }
 
-  TraceScope getPathTraceScope(String description, String path) {
-    TraceScope scope = Trace.startSpan(description, traceSampler);
-    Span span = scope.getSpan();
-    if (span != null) {
-      if (path != null) {
-        span.addKVAnnotation("path", path);
-      }
+  TraceScope newPathTraceScope(String description, String path) {
+    TraceScope scope = tracer.newScope(description);
+    if (path != null) {
+      scope.addKVAnnotation("path", path);
     }
     return scope;
   }
 
-  TraceScope getSrcDstTraceScope(String description, String src, String dst) {
-    TraceScope scope = Trace.startSpan(description, traceSampler);
-    Span span = scope.getSpan();
-    if (span != null) {
-      if (src != null) {
-        span.addKVAnnotation("src", src);
-      }
-      if (dst != null) {
-        span.addKVAnnotation("dst", dst);
-      }
+  TraceScope newSrcDstTraceScope(String description, String src, String dst) {
+    TraceScope scope = tracer.newScope(description);
+    if (src != null) {
+      scope.addKVAnnotation("src", src);
+    }
+    if (dst != null) {
+      scope.addKVAnnotation("dst", dst);
     }
     return scope;
   }
+
+  Tracer getTracer() {
+    return tracer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index 11a1d29..c98cd5f 100644
--- 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
@@ -26,9 +26,8 @@ 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.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,11 +46,6 @@ 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;
@@ -65,20 +59,22 @@ public class DFSInotifyEventInputStream {
    */
   private Random rng = new Random();
 
+  private final Tracer tracer;
+
   private static final int INITIAL_WAIT_MS = 10;
 
-  DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode)
+  DFSInotifyEventInputStream(ClientProtocol namenode, Tracer tracer)
         throws IOException {
     // Only consider new transaction IDs.
-    this(traceSampler, namenode, namenode.getCurrentEditLogTxid());
+    this(namenode, tracer, namenode.getCurrentEditLogTxid());
   }
 
-  DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode,
-        long lastReadTxid) throws IOException {
-    this.traceSampler = traceSampler;
+  DFSInotifyEventInputStream(ClientProtocol namenode,
+        Tracer tracer, long lastReadTxid) throws IOException {
     this.namenode = namenode;
     this.it = Iterators.emptyIterator();
     this.lastReadTxid = lastReadTxid;
+    this.tracer = tracer;
   }
 
   /**
@@ -98,8 +94,7 @@ public class DFSInotifyEventInputStream {
    * The next available batch of events will be returned.
    */
   public EventBatch poll() throws IOException, MissingEventsException {
-    TraceScope scope =
-        Trace.startSpan("inotifyPoll", traceSampler);
+    TraceScope scope = tracer.newScope("inotifyPoll");
     try {
       // need to keep retrying until the NN sends us the latest committed txid
       if (lastReadTxid == -1) {
@@ -180,7 +175,7 @@ public class DFSInotifyEventInputStream {
    */
   public EventBatch poll(long time, TimeUnit tu) throws IOException,
       InterruptedException, MissingEventsException {
-    TraceScope scope = Trace.startSpan("inotifyPollWithTimeout", traceSampler);
+    TraceScope scope = tracer.newScope("inotifyPollWithTimeout");
     EventBatch next = null;
     try {
       long initialTime = Time.monotonicNow();
@@ -217,7 +212,7 @@ public class DFSInotifyEventInputStream {
    */
   public EventBatch take() throws IOException, InterruptedException,
       MissingEventsException {
-    TraceScope scope = Trace.startSpan("inotifyTake", traceSampler);
+    TraceScope scope = tracer.newScope("inotifyTake");
     EventBatch next = null;
     try {
       int nextWaitMin = INITIAL_WAIT_MS;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index 139a27c..7101753 100644
--- 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
@@ -55,6 +55,7 @@ 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.FileSystem;
 import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
@@ -78,9 +79,9 @@ 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 org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -678,6 +679,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         setClientCacheContext(dfsClient.getClientContext()).
         setUserGroupInformation(dfsClient.ugi).
         setConfiguration(dfsClient.getConfiguration()).
+        setTracer(dfsClient.getTracer()).
         build();
   }
 
@@ -941,7 +943,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   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);
+        dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src);
     try {
       return readWithStrategy(byteArrayReader, off, len);
     } finally {
@@ -953,7 +955,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   public synchronized int read(final ByteBuffer buf) throws IOException {
     ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
     TraceScope scope =
-        dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
+        dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src);
     try {
       return readWithStrategy(byteBufferReader, 0, buf.remaining());
     } finally {
@@ -1120,14 +1122,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       final ByteBuffer bb,
       final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
       final int hedgedReadId) {
-    final Span parentSpan = Trace.currentSpan();
+    final SpanId parentSpanId = Tracer.getCurrentSpanId();
     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);
+        TraceScope scope = dfsClient.getTracer().
+            newScope("hedgedRead" + hedgedReadId, parentSpanId);
         try {
           actualGetFromOneDataNode(datanode, block, start, end, buf,
               offset, corruptedBlockMap);
@@ -1449,8 +1451,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   @Override
   public int read(long position, byte[] buffer, int offset, int length)
       throws IOException {
-    TraceScope scope =
-        dfsClient.getPathTraceScope("DFSInputStream#byteArrayPread", src);
+    TraceScope scope = dfsClient.
+        newPathTraceScope("DFSInputStream#byteArrayPread", src);
     try {
       return pread(position, buffer, offset, length);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index de1d1ee..7a40d73 100755
--- 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
@@ -32,6 +32,7 @@ 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.FileSystem;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -62,9 +63,7 @@ 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.apache.htrace.core.TraceScope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -227,7 +226,7 @@ public class DFSOutputStream extends FSOutputSummer
       short replication, long blockSize, Progressable progress, int buffersize,
       DataChecksum checksum, String[] favoredNodes) throws IOException {
     TraceScope scope =
-        dfsClient.getPathTraceScope("newStreamForCreate", src);
+        dfsClient.newPathTraceScope("newStreamForCreate", src);
     try {
       HdfsFileStatus stat = null;
 
@@ -350,7 +349,7 @@ public class DFSOutputStream extends FSOutputSummer
       LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
       String[] favoredNodes) throws IOException {
     TraceScope scope =
-        dfsClient.getPathTraceScope("newStreamForAppend", src);
+        dfsClient.newPathTraceScope("newStreamForAppend", src);
     try {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
           progress, lastBlock, stat, checksum, favoredNodes);
@@ -375,7 +374,7 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   protected TraceScope createWriteTraceScope() {
-    return dfsClient.getPathTraceScope("DFSOutputStream#write", src);
+    return dfsClient.newPathTraceScope("DFSOutputStream#write", src);
   }
 
   // @see FSOutputSummer#writeChunk()
@@ -490,7 +489,7 @@ public class DFSOutputStream extends FSOutputSummer
   @Override
   public void hflush() throws IOException {
     TraceScope scope =
-        dfsClient.getPathTraceScope("hflush", src);
+        dfsClient.newPathTraceScope("hflush", src);
     try {
       flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
     } finally {
@@ -501,7 +500,7 @@ public class DFSOutputStream extends FSOutputSummer
   @Override
   public void hsync() throws IOException {
     TraceScope scope =
-        dfsClient.getPathTraceScope("hsync", src);
+        dfsClient.newPathTraceScope("hsync", src);
     try {
       flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
     } finally {
@@ -524,7 +523,7 @@ public class DFSOutputStream extends FSOutputSummer
    */
   public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
     TraceScope scope =
-        dfsClient.getPathTraceScope("hsync", src);
+        dfsClient.newPathTraceScope("hsync", src);
     try {
       flushOrSync(true, syncFlags);
     } finally {
@@ -765,7 +764,7 @@ public class DFSOutputStream extends FSOutputSummer
   @Override
   public synchronized void close() throws IOException {
     TraceScope scope =
-        dfsClient.getPathTraceScope("DFSOutputStream#close", src);
+        dfsClient.newPathTraceScope("DFSOutputStream#close", src);
     try {
       closeImpl();
     } finally {
@@ -794,7 +793,7 @@ public class DFSOutputStream extends FSOutputSummer
       // get last block before destroying the streamer
       ExtendedBlock lastBlock = getStreamer().getBlock();
       closeThreads(false);
-      TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
+      TraceScope scope = dfsClient.getTracer().newScope("completeFile");
       try {
         completeFile(lastBlock);
       } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index 22055c3..9a8ca6f 100755
--- 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
@@ -27,7 +27,9 @@ 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;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.TraceScope;
 
 /****************************************************************
  * DFSPacket is used by DataStreamer and DFSOutputStream.
@@ -38,7 +40,7 @@ import org.apache.htrace.Span;
 @InterfaceAudience.Private
 class DFSPacket {
   public static final long HEART_BEAT_SEQNO = -1L;
-  private static long[] EMPTY = new long[0];
+  private static SpanId[] EMPTY = new SpanId[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
@@ -65,9 +67,9 @@ class DFSPacket {
   private int checksumPos;
   private final int dataStart;
   private int dataPos;
-  private long[] traceParents = EMPTY;
+  private SpanId[] traceParents = EMPTY;
   private int traceParentsUsed;
-  private Span span;
+  private TraceScope scope;
 
   /**
    * Create a new packet.
@@ -293,7 +295,10 @@ class DFSPacket {
     addTraceParent(span.getSpanId());
   }
 
-  public void addTraceParent(long id) {
+  public void addTraceParent(SpanId id) {
+    if (!id.isValid()) {
+      return;
+    }
     if (traceParentsUsed == traceParents.length) {
       int newLength = (traceParents.length == 0) ? 8 :
           traceParents.length * 2;
@@ -310,18 +315,18 @@ class DFSPacket {
    *
    * Protected by the DFSOutputStream dataQueue lock.
    */
-  public long[] getTraceParents() {
+  public SpanId[] 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
+    SpanId prevVal = SpanId.INVALID;
     while (true) {
       if (i == len) {
         break;
       }
-      long val = traceParents[i];
-      if (val != prevVal) {
+      SpanId val = traceParents[i];
+      if (!val.equals(prevVal)) {
         traceParents[j] = val;
         j++;
         prevVal = val;
@@ -335,11 +340,11 @@ class DFSPacket {
     return traceParents;
   }
 
-  public void setTraceSpan(Span span) {
-    this.span = span;
+  public void setTraceScope(TraceScope scope) {
+    this.scope = scope;
   }
 
-  public Span getTraceSpan() {
-    return span;
+  public TraceScope getTraceScope() {
+    return scope;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index fb57825..6482966 100644
--- 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
@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@@ -79,12 +80,11 @@ 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 org.apache.htrace.core.Sampler;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
@@ -505,7 +505,7 @@ class DataStreamer extends Daemon {
   @Override
   public void run() {
     long lastPacket = Time.monotonicNow();
-    TraceScope scope = NullScope.INSTANCE;
+    TraceScope scope = null;
     while (!streamerClosed && dfsClient.clientRunning) {
       // if the Responder encountered an error, shutdown Responder
       if (errorState.hasError() && response != null) {
@@ -556,12 +556,11 @@ class DataStreamer extends Daemon {
               LOG.warn("Caught exception", e);
             }
             one = dataQueue.getFirst(); // regular data packet
-            long parents[] = one.getTraceParents();
+            SpanId[] 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);
+              scope = dfsClient.getTracer().
+                  newScope("dataStreamer", parents[0]);
+              scope.getSpan().setParents(parents);
             }
           }
         }
@@ -612,12 +611,16 @@ class DataStreamer extends Daemon {
         }
 
         // send the packet
-        Span span = null;
+        SpanId spanId = SpanId.INVALID;
         synchronized (dataQueue) {
           // move packet from dataQueue to ackQueue
           if (!one.isHeartbeatPacket()) {
-            span = scope.detach();
-            one.setTraceSpan(span);
+            if (scope != null) {
+              spanId = scope.getSpanId();
+              scope.detach();
+              one.setTraceScope(scope);
+            }
+            scope = null;
             dataQueue.removeFirst();
             ackQueue.addLast(one);
             dataQueue.notifyAll();
@@ -630,7 +633,8 @@ class DataStreamer extends Daemon {
         }
 
         // write out data to remote datanode
-        TraceScope writeScope = Trace.startSpan("writeTo", span);
+        TraceScope writeScope = dfsClient.getTracer().
+            newScope("DataStreamer#writeTo", spanId);
         try {
           one.writeTo(blockStream);
           blockStream.flush();
@@ -697,7 +701,10 @@ class DataStreamer extends Daemon {
           streamerClosed = true;
         }
       } finally {
-        scope.close();
+        if (scope != null) {
+          scope.close();
+          scope = null;
+        }
       }
     }
     closeInternal();
@@ -731,7 +738,8 @@ class DataStreamer extends Daemon {
    * @throws IOException
    */
   void waitForAckedSeqno(long seqno) throws IOException {
-    TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
+    TraceScope scope = dfsClient.getTracer().
+        newScope("waitForAckedSeqno");
     try {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Waiting for ack for: " + seqno);
@@ -781,7 +789,7 @@ class DataStreamer extends Daemon {
           while (!streamerClosed && dataQueue.size() + ackQueue.size() >
               dfsClient.getConf().getWriteMaxPackets()) {
             if (firstWait) {
-              Span span = Trace.currentSpan();
+              Span span = Tracer.getCurrentSpan();
               if (span != null) {
                 span.addTimelineAnnotation("dataQueue.wait");
               }
@@ -802,7 +810,7 @@ class DataStreamer extends Daemon {
             }
           }
         } finally {
-          Span span = Trace.currentSpan();
+          Span span = Tracer.getCurrentSpan();
           if ((span != null) && (!firstWait)) {
             span.addTimelineAnnotation("end.wait");
           }
@@ -934,7 +942,7 @@ class DataStreamer extends Daemon {
       setName("ResponseProcessor for block " + block);
       PipelineAck ack = new PipelineAck();
 
-      TraceScope scope = NullScope.INSTANCE;
+      TraceScope scope = null;
       while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
         // process responses from datanodes.
         try {
@@ -1021,8 +1029,11 @@ class DataStreamer extends Daemon {
           block.setNumBytes(one.getLastByteOffsetBlock());
 
           synchronized (dataQueue) {
-            scope = Trace.continueSpan(one.getTraceSpan());
-            one.setTraceSpan(null);
+            scope = one.getTraceScope();
+            if (scope != null) {
+              scope.reattach();
+              one.setTraceScope(null);
+            }
             lastAckedSeqno = seqno;
             ackQueue.removeFirst();
             dataQueue.notifyAll();
@@ -1043,7 +1054,10 @@ class DataStreamer extends Daemon {
             responderClosed = true;
           }
         } finally {
+          if (scope != null) {
             scope.close();
+          }
+          scope = null;
         }
       }
     }
@@ -1109,11 +1123,12 @@ class DataStreamer extends Daemon {
         // 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);
+          // Close any trace span associated with this Packet
+          TraceScope scope = endOfBlockPacket.getTraceScope();
+          if (scope != null) {
+            scope.reattach();
             scope.close();
+            endOfBlockPacket.setTraceScope(null);
           }
           assert endOfBlockPacket.isLastPacketInBlock();
           assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
@@ -1741,7 +1756,7 @@ class DataStreamer extends Daemon {
   void queuePacket(DFSPacket packet) {
     synchronized (dataQueue) {
       if (packet == null) return;
-      packet.addTraceParent(Trace.currentSpan());
+      packet.addTraceParent(Tracer.getCurrentSpanId());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
       if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index 7509da5..15a5bee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -47,9 +47,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -106,6 +105,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   int dataLeft = 0;
   
   private final PeerCache peerCache;
+
+  private final Tracer tracer;
   
   /* FSInputChecker interface */
   
@@ -210,9 +211,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   protected synchronized int readChunk(long pos, byte[] buf, int offset, 
                                        int len, byte[] checksumBuf) 
                                        throws IOException {
-    TraceScope scope =
-        Trace.startSpan("RemoteBlockReader#readChunk(" + blockId + ")",
-            Sampler.NEVER);
+    TraceScope scope = tracer.
+        newScope("RemoteBlockReader#readChunk(" + blockId + ")");
     try {
       return readChunkImpl(pos, buf, offset, len, checksumBuf);
     } finally {
@@ -346,7 +346,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   private RemoteBlockReader(String file, String bpid, long blockId,
       DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
-      DatanodeID datanodeID, PeerCache peerCache) {
+      DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
     // Path is used only for printing block and file information in debug
     super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
                     ":" + bpid + ":of:"+ file)/*too non path-like?*/,
@@ -378,6 +378,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     bytesPerChecksum = this.checksum.getBytesPerChecksum();
     checksumSize = this.checksum.getChecksumSize();
     this.peerCache = peerCache;
+    this.tracer = tracer;
   }
 
   /**
@@ -402,7 +403,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
                                      String clientName, Peer peer,
                                      DatanodeID datanodeID,
                                      PeerCache peerCache,
-                                     CachingStrategy cachingStrategy)
+                                     CachingStrategy cachingStrategy,
+                                     Tracer tracer)
                                        throws IOException {
     // in and out will be closed when sock is closed (by the caller)
     final DataOutputStream out =
@@ -438,7 +440,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
     return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
         in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
-        peer, datanodeID, peerCache);
+        peer, datanodeID, peerCache, tracer);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 5541e6d..7a7932d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -48,12 +48,11 @@ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.htrace.core.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -126,6 +125,8 @@ public class RemoteBlockReader2  implements BlockReader {
 
   private boolean sentStatusCode = false;
 
+  private final Tracer tracer;
+
   @VisibleForTesting
   public Peer getPeer() {
     return peer;
@@ -144,8 +145,8 @@ public class RemoteBlockReader2  implements BlockReader {
     }
 
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
-      TraceScope scope = Trace.startSpan(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
+      TraceScope scope = tracer.newScope(
+          "RemoteBlockReader2#readNextPacket(" + blockId + ")");
       try {
         readNextPacket();
       } finally {
@@ -172,8 +173,8 @@ public class RemoteBlockReader2  implements BlockReader {
   @Override
   public synchronized int read(ByteBuffer buf) throws IOException {
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
-      TraceScope scope = Trace.startSpan(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
+      TraceScope scope = tracer.newScope(
+          "RemoteBlockReader2#readNextPacket(" + blockId + ")");
       try {
         readNextPacket();
       } finally {
@@ -292,7 +293,7 @@ public class RemoteBlockReader2  implements BlockReader {
   protected RemoteBlockReader2(String file, String bpid, long blockId,
       DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
-      DatanodeID datanodeID, PeerCache peerCache) {
+      DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
     this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
         createSocketAddr(datanodeID.getXferAddr()));
     // Path is used only for printing block and file information in debug
@@ -313,6 +314,7 @@ public class RemoteBlockReader2  implements BlockReader {
     this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
     bytesPerChecksum = this.checksum.getBytesPerChecksum();
     checksumSize = this.checksum.getChecksumSize();
+    this.tracer = tracer;
   }
 
 
@@ -407,7 +409,8 @@ public class RemoteBlockReader2  implements BlockReader {
                                      String clientName,
                                      Peer peer, DatanodeID datanodeID,
                                      PeerCache peerCache,
-                                     CachingStrategy cachingStrategy) throws IOException {
+                                     CachingStrategy cachingStrategy,
+                                     Tracer tracer) throws IOException {
     // in and out will be closed when sock is closed (by the caller)
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
           peer.getOutputStream()));
@@ -440,7 +443,7 @@ public class RemoteBlockReader2  implements BlockReader {
 
     return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
         checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
-        datanodeID, peerCache);
+        datanodeID, peerCache, tracer);
   }
 
   static void checkSuccess(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index 923cdb4..f144a55 100644
--- 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
@@ -25,11 +25,10 @@ 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;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 /**
  * CacheDirectiveIterator is a remote iterator that iterates cache directives.
@@ -42,14 +41,14 @@ public class CacheDirectiveIterator
 
   private CacheDirectiveInfo filter;
   private final ClientProtocol namenode;
-  private final Sampler<?> traceSampler;
+  private final Tracer tracer;
 
   public CacheDirectiveIterator(ClientProtocol namenode,
-      CacheDirectiveInfo filter, Sampler<?> traceSampler) {
+      CacheDirectiveInfo filter, Tracer tracer) {
     super(0L);
     this.namenode = namenode;
     this.filter = filter;
-    this.traceSampler = traceSampler;
+    this.tracer = tracer;
   }
 
   private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
@@ -94,7 +93,7 @@ public class CacheDirectiveIterator
   public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
       throws IOException {
     BatchedEntries<CacheDirectiveEntry> entries = null;
-    TraceScope scope = Trace.startSpan("listCacheDirectives", traceSampler);
+    TraceScope scope = tracer.newScope("listCacheDirectives");
     try {
       entries = namenode.listCacheDirectives(prevKey, filter);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index e9481f7..5e2bbf2 100644
--- 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
@@ -23,9 +23,8 @@ 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;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 /**
  * CachePoolIterator is a remote iterator that iterates cache pools.
@@ -37,18 +36,18 @@ public class CachePoolIterator
     extends BatchedRemoteIterator<String, CachePoolEntry> {
 
   private final ClientProtocol namenode;
-  private final Sampler traceSampler;
+  private final Tracer tracer;
 
-  public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) {
+  public CachePoolIterator(ClientProtocol namenode, Tracer tracer) {
     super("");
     this.namenode = namenode;
-    this.traceSampler = traceSampler;
+    this.tracer = tracer;
   }
 
   @Override
   public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
       throws IOException {
-    TraceScope scope = Trace.startSpan("listCachePools", traceSampler);
+    TraceScope scope = tracer.newScope("listCachePools");
     try {
       return namenode.listCachePools(prevKey);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index 0141215..a3cff82 100644
--- 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
@@ -23,9 +23,8 @@ 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;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 /**
  * EncryptionZoneIterator is a remote iterator that iterates over encryption
@@ -37,19 +36,18 @@ public class EncryptionZoneIterator
     extends BatchedRemoteIterator<Long, EncryptionZone> {
 
   private final ClientProtocol namenode;
-  private final Sampler<?> traceSampler;
+  private final Tracer tracer;
 
-  public EncryptionZoneIterator(ClientProtocol namenode,
-                                Sampler<?> traceSampler) {
+  public EncryptionZoneIterator(ClientProtocol namenode, Tracer tracer) {
     super(Long.valueOf(0));
     this.namenode = namenode;
-    this.traceSampler = traceSampler;
+    this.tracer = tracer;
   }
 
   @Override
   public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
       throws IOException {
-    TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler);
+    TraceScope scope = tracer.newScope("listEncryptionZones");
     try {
       return namenode.listEncryptionZones(prevId);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
index e2e5f39..e585328 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
@@ -35,10 +35,8 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceInfo;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.Tracer;
 
 /**
  * Static utilities for dealing with the protocol buffers used by the
@@ -89,39 +87,21 @@ public abstract class DataTransferProtoUtil {
     BaseHeaderProto.Builder builder =  BaseHeaderProto.newBuilder()
       .setBlock(PBHelperClient.convert(blk))
       .setToken(PBHelperClient.convert(blockToken));
-    if (Trace.isTracing()) {
-      Span s = Trace.currentSpan();
+    SpanId spanId = Tracer.getCurrentSpanId();
+    if (spanId.isValid()) {
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()
-          .setTraceId(s.getTraceId())
-          .setParentId(s.getSpanId()));
+          .setTraceId(spanId.getHigh())
+          .setParentId(spanId.getLow()));
     }
     return builder.build();
   }
 
-  public static TraceInfo fromProto(DataTransferTraceInfoProto proto) {
-    if (proto == null) return null;
-    if (!proto.hasTraceId()) return null;
-    return new TraceInfo(proto.getTraceId(), proto.getParentId());
-  }
-
-  public static TraceScope continueTraceSpan(ClientOperationHeaderProto header,
-      String description) {
-    return continueTraceSpan(header.getBaseHeader(), description);
-  }
-
-  public static TraceScope continueTraceSpan(BaseHeaderProto header,
-      String description) {
-    return continueTraceSpan(header.getTraceInfo(), description);
-  }
-
-  public static TraceScope continueTraceSpan(DataTransferTraceInfoProto proto,
-      String description) {
-    TraceScope scope = null;
-    TraceInfo info = fromProto(proto);
-    if (info != null) {
-      scope = Trace.startSpan(description, info);
+  public static SpanId fromProto(DataTransferTraceInfoProto proto) {
+    if ((proto != null) && proto.hasTraceId() &&
+          proto.hasParentId()) {
+      return new SpanId(proto.getTraceId(), proto.getParentId());
     }
-    return scope;
+    return null;
   }
 
   public static void checkBlockOpStatus(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index 2d11dc2..e856211 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -48,8 +48,8 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
-import org.apache.htrace.Trace;
-import org.apache.htrace.Span;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.Tracer;
 
 import com.google.protobuf.Message;
 
@@ -200,10 +200,11 @@ public class Sender implements DataTransferProtocol {
     ReleaseShortCircuitAccessRequestProto.Builder builder =
         ReleaseShortCircuitAccessRequestProto.newBuilder().
         setSlotId(PBHelperClient.convert(slotId));
-    if (Trace.isTracing()) {
-      Span s = Trace.currentSpan();
-      builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()
-          .setTraceId(s.getTraceId()).setParentId(s.getSpanId()));
+    SpanId spanId = Tracer.getCurrentSpanId();
+    if (spanId.isValid()) {
+      builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
+          setTraceId(spanId.getHigh()).
+          setParentId(spanId.getLow()));
     }
     ReleaseShortCircuitAccessRequestProto proto = builder.build();
     send(out, Op.RELEASE_SHORT_CIRCUIT_FDS, proto);
@@ -214,10 +215,11 @@ public class Sender implements DataTransferProtocol {
     ShortCircuitShmRequestProto.Builder builder =
         ShortCircuitShmRequestProto.newBuilder().
         setClientName(clientName);
-    if (Trace.isTracing()) {
-      Span s = Trace.currentSpan();
-      builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()
-          .setTraceId(s.getTraceId()).setParentId(s.getSpanId()));
+    SpanId spanId = Tracer.getCurrentSpanId();
+    if (spanId.isValid()) {
+      builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
+          setTraceId(spanId.getHigh()).
+          setParentId(spanId.getLow()));
     }
     ShortCircuitShmRequestProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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 b934d2f..3571e4a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -986,6 +986,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8740. Move DistributedFileSystem to hadoop-hdfs-client. (Mingliang Liu
     via wheat9)
 
+    HDFS-9080. Update htrace version to 4.0.1 (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 6a93331..ef3bcbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -193,7 +193,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     </dependency>
     <dependency>
       <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core</artifactId>
+      <artifactId>htrace-core4</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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 f289b32..b631955 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
@@ -64,12 +64,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
 
-  // HDFS HTrace configuration is controlled by dfs.htrace.spanreceiver.classes,
-  // etc.
-  public static final String  DFS_SERVER_HTRACE_PREFIX = "dfs.htrace.";
-  @Deprecated
-  public static final String  DFS_CLIENT_HTRACE_PREFIX =
-      HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX;
+  public static final String DFS_CLIENT_HTRACE_SAMPLER_CLASSES =
+      "dfs.client.htrace.sampler.classes";
 
   // HA related configuration
   public static final String  DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 59cf884..e040157 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
 import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto;
-import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.continueTraceSpan;
 import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
 
 import java.io.DataInputStream;
@@ -27,7 +26,10 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
@@ -40,14 +42,21 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmR
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 /** Receiver */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public abstract class Receiver implements DataTransferProtocol {
+  private final Tracer tracer;
   protected DataInputStream in;
-  
+
+  protected Receiver(Tracer tracer) {
+    this.tracer = tracer;
+  }
+
   /** Initialize a receiver for DataTransferProtocol with a socket. */
   protected void initialize(final DataInputStream in) {
     this.in = in;
@@ -64,6 +73,26 @@ public abstract class Receiver implements DataTransferProtocol {
     return Op.read(in);
   }
 
+  private TraceScope continueTraceSpan(DataTransferTraceInfoProto proto,
+                                       String description) {
+    TraceScope scope = null;
+    SpanId spanId = fromProto(proto);
+    if (spanId != null) {
+      scope = tracer.newScope(description, spanId);
+    }
+    return scope;
+  }
+
+  private TraceScope continueTraceSpan(ClientOperationHeaderProto header,
+                                             String description) {
+    return continueTraceSpan(header.getBaseHeader(), description);
+  }
+
+  private TraceScope continueTraceSpan(BaseHeaderProto header,
+                                             String description) {
+    return continueTraceSpan(header.getTraceInfo(), description);
+  }
+
   /** Process op by the corresponding method. */
   protected final void processOp(Op op) throws IOException {
     switch(op) {


[43/58] [abbrv] hadoop git commit: YARN-4066. Large number of queues choke fair scheduler. (Johan Gustavsson via kasha)

Posted by zh...@apache.org.
YARN-4066. Large number of queues choke fair scheduler. (Johan Gustavsson via kasha)


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

Branch: refs/heads/HDFS-7285
Commit: a0b5a0a419dfc07b7ac45c06b11b4c8dc7e79958
Parents: 715dbdd
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Sep 29 07:55:34 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Tue Sep 29 07:55:34 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../scheduler/fair/QueueManager.java            | 80 ++++++++++++++------
 2 files changed, 60 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0b5a0a4/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e9d04d3..43c501f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -509,6 +509,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3635. Refactored current queue mapping implementation in CapacityScheduler
     to use a generic PlacementManager framework. (Wangda Tan via jianhe)
 
+    YARN-4066. Large number of queues choke fair scheduler.
+    (Johan Gustavsson via kasha)
+
   BUG FIXES
 
     YARN-3197. Confusing log generated by CapacityScheduler. (Varun Saxena 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0b5a0a4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
index 0092845..51a298b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
@@ -87,7 +87,19 @@ public class QueueManager {
    * could be referred to as just "parent1.queue2".
    */
   public FSLeafQueue getLeafQueue(String name, boolean create) {
-    FSQueue queue = getQueue(name, create, FSQueueType.LEAF);
+    return getLeafQueue(name, create, true);
+  }
+
+  public FSLeafQueue getLeafQueue(
+      String name,
+      boolean create,
+      boolean recomputeSteadyShares) {
+    FSQueue queue = getQueue(
+        name,
+        create,
+        FSQueueType.LEAF,
+        recomputeSteadyShares
+    );
     if (queue instanceof FSParentQueue) {
       return null;
     }
@@ -117,28 +129,46 @@ public class QueueManager {
    * could be referred to as just "parent1.queue2".
    */
   public FSParentQueue getParentQueue(String name, boolean create) {
-    FSQueue queue = getQueue(name, create, FSQueueType.PARENT);
+    return getParentQueue(name, create, true);
+  }
+
+  public FSParentQueue getParentQueue(
+      String name,
+      boolean create,
+      boolean recomputeSteadyShares) {
+    FSQueue queue = getQueue(
+        name,
+        create,
+        FSQueueType.PARENT,
+        recomputeSteadyShares
+    );
     if (queue instanceof FSLeafQueue) {
       return null;
     }
     return (FSParentQueue) queue;
   }
-  
-  private FSQueue getQueue(String name, boolean create, FSQueueType queueType) {
+
+  private FSQueue getQueue(
+      String name,
+      boolean create,
+      FSQueueType queueType,
+      boolean recomputeSteadyShares) {
+    boolean recompute = recomputeSteadyShares;
     name = ensureRootPrefix(name);
+    FSQueue queue;
     synchronized (queues) {
-      FSQueue queue = queues.get(name);
+      queue = queues.get(name);
       if (queue == null && create) {
         // if the queue doesn't exist,create it and return
         queue = createQueue(name, queueType);
-
-        // Update steady fair share for all queues
-        if (queue != null) {
-          rootQueue.recomputeSteadyShares();
-        }
+      } else {
+        recompute = false;
       }
-      return queue;
     }
+    if (recompute) {
+      rootQueue.recomputeSteadyShares();
+    }
+    return queue;
   }
   
   /**
@@ -376,21 +406,25 @@ public class QueueManager {
   
   public void updateAllocationConfiguration(AllocationConfiguration queueConf) {
     // Create leaf queues and the parent queues in a leaf's ancestry if they do not exist
-    for (String name : queueConf.getConfiguredQueues().get(FSQueueType.LEAF)) {
-      if (removeEmptyIncompatibleQueues(name, FSQueueType.LEAF)) {
-        getLeafQueue(name, true);
+    synchronized (queues) {
+      for (String name : queueConf.getConfiguredQueues().get(
+              FSQueueType.LEAF)) {
+        if (removeEmptyIncompatibleQueues(name, FSQueueType.LEAF)) {
+          getLeafQueue(name, true, false);
+        }
       }
-    }
-
-    // At this point all leaves and 'parents with at least one child' would have been created.
-    // Now create parents with no configured leaf.
-    for (String name : queueConf.getConfiguredQueues().get(
-        FSQueueType.PARENT)) {
-      if (removeEmptyIncompatibleQueues(name, FSQueueType.PARENT)) {
-        getParentQueue(name, true);
+      // At this point all leaves and 'parents with
+      // at least one child' would have been created.
+      // Now create parents with no configured leaf.
+      for (String name : queueConf.getConfiguredQueues().get(
+          FSQueueType.PARENT)) {
+        if (removeEmptyIncompatibleQueues(name, FSQueueType.PARENT)) {
+          getParentQueue(name, true, false);
+        }
       }
     }
-    
+    rootQueue.recomputeSteadyShares();
+
     for (FSQueue queue : queues.values()) {
       // Update queue metrics
       FSQueueMetrics queueMetrics = queue.getMetrics();


[28/58] [abbrv] hadoop git commit: YARN-4180. AMLauncher does not retry on failures when talking to NM. (adhoot)

Posted by zh...@apache.org.
YARN-4180. AMLauncher does not retry on failures when talking to NM. (adhoot)


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

Branch: refs/heads/HDFS-7285
Commit: 9735afe967a660f356e953348cb6c34417f41055
Parents: 9f53a95
Author: Anubhav Dhoot <ad...@apache.org>
Authored: Mon Sep 28 15:30:17 2015 -0700
Committer: Anubhav Dhoot <ad...@apache.org>
Committed: Mon Sep 28 16:13:41 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../resourcemanager/amlauncher/AMLauncher.java  | 27 ++++----
 .../yarn/server/resourcemanager/MockRM.java     | 12 ++--
 .../TestApplicationMasterLauncher.java          | 66 ++++++++++++++++++++
 4 files changed, 89 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9735afe9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3745d55..e9d04d3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -999,6 +999,9 @@ Release 2.7.2 - UNRELEASED
     YARN-3624. ApplicationHistoryServer should not reverse the order of the
     filters it gets. (Mit Desai via xgong)
 
+    YARN-4180. AMLauncher does not retry on failures when talking to NM.
+    (adhoot)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9735afe9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
index 713e75f..b1d8506 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.amlauncher;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -51,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.client.NMProxy;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -151,10 +151,10 @@ public class AMLauncher implements Runnable {
       final ContainerId containerId) {
 
     final NodeId node = masterContainer.getNodeId();
-    final InetSocketAddress containerManagerBindAddress =
+    final InetSocketAddress containerManagerConnectAddress =
         NetUtils.createSocketAddrForHost(node.getHost(), node.getPort());
 
-    final YarnRPC rpc = YarnRPC.create(conf); // TODO: Don't create again and again.
+    final YarnRPC rpc = getYarnRPC();
 
     UserGroupInformation currentUser =
         UserGroupInformation.createRemoteUser(containerId
@@ -168,18 +168,15 @@ public class AMLauncher implements Runnable {
         rmContext.getNMTokenSecretManager().createNMToken(
             containerId.getApplicationAttemptId(), node, user);
     currentUser.addToken(ConverterUtils.convertFromYarn(token,
-        containerManagerBindAddress));
-
-    return currentUser
-        .doAs(new PrivilegedAction<ContainerManagementProtocol>() {
-
-          @Override
-          public ContainerManagementProtocol run() {
-            return (ContainerManagementProtocol) rpc.getProxy(
-                ContainerManagementProtocol.class,
-                containerManagerBindAddress, conf);
-          }
-        });
+        containerManagerConnectAddress));
+
+    return NMProxy.createNMProxy(conf, ContainerManagementProtocol.class,
+        currentUser, rpc, containerManagerConnectAddress);
+  }
+
+  @VisibleForTesting
+  protected YarnRPC getYarnRPC() {
+    return YarnRPC.create(conf);  // TODO: Don't create again and again.
   }
 
   private ContainerLaunchContext createAMContainerLaunchContext(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9735afe9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 7ce42f5..a066ba4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -160,14 +160,18 @@ public class MockRM extends ResourceManager {
           " for the application " + appId);
     }
   }
-  
-  public void waitForState(ApplicationAttemptId attemptId, 
-                           RMAppAttemptState finalState)
+
+  public void waitForState(ApplicationAttemptId attemptId,
+      RMAppAttemptState finalState)
       throws Exception {
+    waitForState(attemptId, finalState, 40000);
+  }
+
+  public void waitForState(ApplicationAttemptId attemptId,
+      RMAppAttemptState finalState, int timeoutMsecs) throws Exception {
     RMApp app = getRMContext().getRMApps().get(attemptId.getApplicationId());
     Assert.assertNotNull("app shouldn't be null", app);
     RMAppAttempt attempt = app.getRMAppAttempt(attemptId);
-    final int timeoutMsecs = 40000;
     final int minWaitMsecs = 1000;
     final int waitMsPerLoop = 10;
     int loop = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9735afe9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index 2760705..8fa88d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -49,14 +50,19 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -67,6 +73,10 @@ import org.apache.log4j.Logger;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class TestApplicationMasterLauncher {
 
   private static final Log LOG = LogFactory
@@ -193,6 +203,62 @@ public class TestApplicationMasterLauncher {
     rm.stop();
   }
 
+  @Test
+  public void testRetriesOnFailures() throws Exception {
+    final ContainerManagementProtocol mockProxy =
+        mock(ContainerManagementProtocol.class);
+    final StartContainersResponse mockResponse =
+        mock(StartContainersResponse.class);
+    when(mockProxy.startContainers(any(StartContainersRequest.class)))
+        .thenThrow(new NMNotYetReadyException("foo")).thenReturn(mockResponse);
+    Configuration conf = new Configuration();
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    conf.setInt(YarnConfiguration.CLIENT_NM_CONNECT_RETRY_INTERVAL_MS, 1);
+    final DrainDispatcher dispatcher = new DrainDispatcher();
+    MockRM rm = new MockRMWithCustomAMLauncher(conf, null) {
+      @Override
+      protected ApplicationMasterLauncher createAMLauncher() {
+        return new ApplicationMasterLauncher(getRMContext()) {
+          @Override
+          protected Runnable createRunnableLauncher(RMAppAttempt application,
+              AMLauncherEventType event) {
+            return new AMLauncher(context, application, event, getConfig()) {
+              @Override
+              protected YarnRPC getYarnRPC() {
+                YarnRPC mockRpc = mock(YarnRPC.class);
+
+                when(mockRpc.getProxy(
+                    any(Class.class),
+                    any(InetSocketAddress.class),
+                    any(Configuration.class)))
+                    .thenReturn(mockProxy);
+                return mockRpc;
+              }
+            };
+          }
+        };
+      }
+
+      @Override
+      protected Dispatcher createDispatcher() {
+        return dispatcher;
+      }
+    };
+    rm.start();
+    MockNM nm1 = rm.registerNode("127.0.0.1:1234", 5120);
+
+    RMApp app = rm.submitApp(2000);
+    final ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
+        .getAppAttemptId();
+
+    // kick the scheduling
+    nm1.nodeHeartbeat(true);
+    dispatcher.await();
+
+    rm.waitForState(appAttemptId, RMAppAttemptState.LAUNCHED, 500);
+  }
+
+
 
   @SuppressWarnings("unused")
   @Test(timeout = 100000)


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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 de1d1ee..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ /dev/null
@@ -1,918 +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;
-    }
-  }
-  
-  /**
-   * 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/bf37d3d8/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/bf37d3d8/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 fe9e342..5b11ac2 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
@@ -67,7 +67,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -1441,27 +1440,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();
-  }
-
 }


[22/58] [abbrv] hadoop git commit: HDFS-9106. Transfer failure during pipeline recovery causes permanent write failures. Contributed by Kihwal Lee.

Posted by zh...@apache.org.
HDFS-9106. Transfer failure during pipeline recovery causes permanent write failures. Contributed by Kihwal Lee.


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

Branch: refs/heads/HDFS-7285
Commit: 4c9497cbf02ecc82532a4e79e18912d8e0eb4731
Parents: fb2e525
Author: Kihwal Lee <ki...@apache.org>
Authored: Mon Sep 28 13:29:19 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Mon Sep 28 13:29:56 2015 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 60 ++++++++++++++------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 2 files changed, 47 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c9497cb/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
index 6482966..d1d8d37 100644
--- 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
@@ -1208,22 +1208,46 @@ class DataStreamer extends Daemon {
       return;
     }
 
-    //get a new datanode
+    int tried = 0;
     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());
+    final StorageType[] originalTypes = storageTypes;
+    final String[] originalIDs = storageIDs;
+    IOException caughtException = null;
+    ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
+    while (tried < 3) {
+      LocatedBlock lb;
+      //get a new datanode
+      lb = dfsClient.namenode.getAdditionalDatanode(
+          src, stat.getFileId(), block, nodes, storageIDs,
+          exclude.toArray(new DatanodeInfo[exclude.size()]),
+          1, dfsClient.clientName);
+      // a new node was allocated by the namenode. Update nodes.
+      setPipeline(lb);
+
+      //find the new datanode
+      final int d = findNewDatanode(original);
+      //transfer replica. pick a source from the original nodes
+      final DatanodeInfo src = original[tried % original.length];
+      final DatanodeInfo[] targets = {nodes[d]};
+      final StorageType[] targetStorageTypes = {storageTypes[d]};
+
+      try {
+        transfer(src, targets, targetStorageTypes, lb.getBlockToken());
+      } catch (IOException ioe) {
+        DFSClient.LOG.warn("Error transferring data from " + src + " to " +
+            nodes[d] + ": " + ioe.getMessage());
+        caughtException = ioe;
+        // add the allocated node to the exclude list.
+        exclude.add(nodes[d]);
+        setPipeline(original, originalTypes, originalIDs);
+        tried++;
+        continue;
+      }
+      return; // finished successfully
+    }
+    // All retries failed
+    throw (caughtException != null) ? caughtException :
+        new IOException("Failed to add a node");
   }
 
   private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
@@ -1236,7 +1260,11 @@ class DataStreamer extends Daemon {
     try {
       sock = createSocketForPipeline(src, 2, dfsClient);
       final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
-      final long readTimeout = dfsClient.getDatanodeReadTimeout(2);
+
+      // transfer timeout multiplier based on the transfer size
+      // One per 200 packets = 12.8MB. Minimum is 2.
+      int multi = 2 + (int)(bytesSent/dfsClient.getConf().getWritePacketSize())/200;
+      final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
 
       OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
       InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c9497cb/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 3571e4a..1d9fa1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1488,6 +1488,9 @@ Release 2.7.2 - UNRELEASED
     HDFS-9043. Doc updation for commands in HDFS Federation
     (J.Andreina via vinayakumab)
 
+    HDFS-9106. Transfer failure during pipeline recovery causes permanent
+    write failures (kihwal)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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;
-  }
-}


[46/58] [abbrv] hadoop git commit: HDFS-9165. Move entries in META-INF/services/o.a.h.fs.FileSystem to hdfs-client. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HDFS-9165. Move entries in META-INF/services/o.a.h.fs.FileSystem to 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/80d33b58
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/80d33b58
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/80d33b58

Branch: refs/heads/HDFS-7285
Commit: 80d33b589b0683f8343575416d77c64af343c5f7
Parents: a2efb94
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 29 09:50:04 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 29 09:50:04 2015 -0700

----------------------------------------------------------------------
 .../services/org.apache.hadoop.fs.FileSystem        |  1 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt         |  3 +++
 .../services/org.apache.hadoop.fs.FileSystem        | 16 ----------------
 3 files changed, 4 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/80d33b58/hadoop-hdfs-project/hadoop-hdfs-client/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
index 3c832de..abe2bfc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
@@ -13,5 +13,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+org.apache.hadoop.hdfs.DistributedFileSystem
 org.apache.hadoop.hdfs.web.WebHdfsFileSystem
 org.apache.hadoop.hdfs.web.SWebHdfsFileSystem

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80d33b58/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 2c90b23..0876727 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -994,6 +994,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8859. Improve DataNode ReplicaMap memory footprint to save about 45%.
     (yliu)
 
+    HDFS-9165. Move entries in META-INF/services/o.a.h.fs.FileSystem to
+    hdfs-client. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80d33b58/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
deleted file mode 100644
index 120ff94..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.hadoop.hdfs.DistributedFileSystem


[32/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index dabae2c,0000000..7c64b37
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@@ -1,1014 -1,0 +1,1016 @@@
 +/**
 + * 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.erasurecode;
 +
 +import java.io.BufferedOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.net.InetSocketAddress;
 +import java.net.Socket;
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.BitSet;
 +import java.util.Collection;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.concurrent.Callable;
 +import java.util.concurrent.CompletionService;
 +import java.util.concurrent.ExecutorCompletionService;
 +import java.util.concurrent.Future;
 +import java.util.concurrent.LinkedBlockingQueue;
 +import java.util.concurrent.SynchronousQueue;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.StorageType;
 +import org.apache.hadoop.hdfs.BlockReader;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSPacket;
 +import org.apache.hadoop.hdfs.DFSUtil;
 +import org.apache.hadoop.hdfs.DFSUtilClient;
 +import org.apache.hadoop.hdfs.RemoteBlockReader2;
 +import org.apache.hadoop.hdfs.net.Peer;
 +import org.apache.hadoop.hdfs.net.TcpPeerServer;
 +import org.apache.hadoop.hdfs.protocol.DatanodeID;
 +import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 +import org.apache.hadoop.hdfs.server.datanode.DataNode;
 +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.erasurecode.CodecUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 +import org.apache.hadoop.net.NetUtils;
 +import org.apache.hadoop.security.token.Token;
 +import org.apache.hadoop.util.Daemon;
 +import org.apache.hadoop.util.DataChecksum;
 +
 +import com.google.common.base.Preconditions;
 +
 +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode;
 +
 +/**
 + * ErasureCodingWorker handles the erasure coding recovery work commands. These
 + * commands would be issued from Namenode as part of Datanode's heart beat
 + * response. BPOfferService delegates the work to this class for handling EC
 + * commands.
 + */
 +public final class ErasureCodingWorker {
 +  private static final Log LOG = DataNode.LOG;
 +  
 +  private final DataNode datanode; 
 +  private final Configuration conf;
 +
 +  private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
 +  private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
 +  private final int STRIPED_READ_TIMEOUT_MILLIS;
 +  private final int STRIPED_READ_BUFFER_SIZE;
 +
 +  public ErasureCodingWorker(Configuration conf, DataNode datanode) {
 +    this.datanode = datanode;
 +    this.conf = conf;
 +
 +    STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
 +    initializeStripedReadThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, 
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT));
 +    STRIPED_READ_BUFFER_SIZE = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
 +
 +    initializeStripedBlkRecoveryThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT));
 +  }
 +  
 +  private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
 +    return CodecUtil.createRSRawDecoder(conf, numDataUnits, numParityUnits);
 +  }
 +
 +  private void initializeStripedReadThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped reads; pool threads=" + num);
 +    }
 +    STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
 +        TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +      private final AtomicInteger threadIndex = new AtomicInteger(0);
 +
 +      @Override
 +      public Thread newThread(Runnable r) {
 +        Thread t = super.newThread(r);
 +        t.setName("stripedRead-" + threadIndex.getAndIncrement());
 +        return t;
 +      }
 +    }, new ThreadPoolExecutor.CallerRunsPolicy() {
 +      @Override
 +      public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
 +        LOG.info("Execution for striped reading rejected, "
 +            + "Executing in current thread");
 +        // will run in the current thread
 +        super.rejectedExecution(runnable, e);
 +      }
 +    });
 +    STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  private void initializeStripedBlkRecoveryThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped block recovery; pool threads=" + num);
 +    }
 +    STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
 +        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +          private final AtomicInteger threadIdx = new AtomicInteger(0);
 +
 +          @Override
 +          public Thread newThread(Runnable r) {
 +            Thread t = super.newThread(r);
 +            t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement());
 +            return t;
 +          }
 +        });
 +    STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  /**
 +   * Handles the Erasure Coding recovery work commands.
 +   * 
 +   * @param ecTasks
 +   *          BlockECRecoveryInfo
 +   */
 +  public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
 +    for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
 +      try {
 +        STRIPED_BLK_RECOVERY_THREAD_POOL
 +            .submit(new ReconstructAndTransferBlock(recoveryInfo));
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block "
 +            + recoveryInfo.getExtendedBlock().getLocalBlock(), e);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * ReconstructAndTransferBlock recover one or more missed striped block in the
 +   * striped block group, the minimum number of live striped blocks should be
 +   * no less than data block number.
 +   * 
 +   * | <- Striped Block Group -> |
 +   *  blk_0      blk_1       blk_2(*)   blk_3   ...   <- A striped block group
 +   *    |          |           |          |  
 +   *    v          v           v          v 
 +   * +------+   +------+   +------+   +------+
 +   * |cell_0|   |cell_1|   |cell_2|   |cell_3|  ...    
 +   * +------+   +------+   +------+   +------+     
 +   * |cell_4|   |cell_5|   |cell_6|   |cell_7|  ...
 +   * +------+   +------+   +------+   +------+
 +   * |cell_8|   |cell_9|   |cell10|   |cell11|  ...
 +   * +------+   +------+   +------+   +------+
 +   *  ...         ...       ...         ...
 +   *  
 +   * 
 +   * We use following steps to recover striped block group, in each round, we
 +   * recover <code>bufferSize</code> data until finish, the 
 +   * <code>bufferSize</code> is configurable and may be less or larger than 
 +   * cell size:
 +   * step1: read <code>bufferSize</code> data from minimum number of sources 
 +   *        required by recovery.
 +   * step2: decode data for targets.
 +   * step3: transfer data to targets.
 +   * 
 +   * In step1, try to read <code>bufferSize</code> data from minimum number
 +   * of sources , if there is corrupt or stale sources, read from new source
 +   * will be scheduled. The best sources are remembered for next round and 
 +   * may be updated in each round.
 +   * 
 +   * In step2, typically if source blocks we read are all data blocks, we 
 +   * need to call encode, and if there is one parity block, we need to call
 +   * decode. Notice we only read once and recover all missed striped block 
 +   * if they are more than one.
 +   * 
 +   * In step3, send the recovered data to targets by constructing packet 
 +   * and send them directly. Same as continuous block replication, we 
 +   * don't check the packet ack. Since the datanode doing the recovery work
 +   * are one of the source datanodes, so the recovered data are sent 
 +   * remotely.
 +   * 
 +   * There are some points we can do further improvements in next phase:
 +   * 1. we can read the block file directly on the local datanode, 
 +   *    currently we use remote block reader. (Notice short-circuit is not
 +   *    a good choice, see inline comments).
 +   * 2. We need to check the packet ack for EC recovery? Since EC recovery
 +   *    is more expensive than continuous block replication, it needs to 
 +   *    read from several other datanodes, should we make sure the 
 +   *    recovered result received by targets? 
 +   */
 +  private class ReconstructAndTransferBlock implements Runnable {
 +    private final int dataBlkNum;
 +    private final int parityBlkNum;
 +    private final int cellSize;
 +    
 +    private RawErasureDecoder decoder;
 +
 +    // Striped read buffer size
 +    private int bufferSize;
 +
 +    private final ExtendedBlock blockGroup;
 +    private final int minRequiredSources;
 +    // position in striped internal block
 +    private long positionInBlock;
 +
 +    // sources
 +    private final short[] liveIndices;
 +    private final DatanodeInfo[] sources;
 +
 +    private final List<StripedReader> stripedReaders;
 +
 +    // The buffers and indices for striped blocks whose length is 0
 +    private ByteBuffer[] zeroStripeBuffers;
 +    private short[] zeroStripeIndices;
 +
 +    // targets
 +    private final DatanodeInfo[] targets;
 +    private final StorageType[] targetStorageTypes;
 +
 +    private final short[] targetIndices;
 +    private final ByteBuffer[] targetBuffers;
 +
 +    private final Socket[] targetSockets;
 +    private final DataOutputStream[] targetOutputStreams;
 +    private final DataInputStream[] targetInputStreams;
 +
 +    private final long[] blockOffset4Targets;
 +    private final long[] seqNo4Targets;
 +
 +    private final static int WRITE_PACKET_SIZE = 64 * 1024;
 +    private DataChecksum checksum;
 +    private int maxChunksPerPacket;
 +    private byte[] packetBuf;
 +    private byte[] checksumBuf;
 +    private int bytesPerChecksum;
 +    private int checksumSize;
 +
 +    private final CachingStrategy cachingStrategy;
 +
 +    private final Map<Future<Void>, Integer> futures = new HashMap<>();
 +    private final CompletionService<Void> readService =
 +        new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
 +
 +    ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
 +      ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy();
 +      dataBlkNum = ecPolicy.getNumDataUnits();
 +      parityBlkNum = ecPolicy.getNumParityUnits();
 +      cellSize = ecPolicy.getCellSize();
 +
 +      blockGroup = recoveryInfo.getExtendedBlock();
 +      final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1);
 +      minRequiredSources = Math.min(cellsNum, dataBlkNum);
 +
 +      liveIndices = recoveryInfo.getLiveBlockIndices();
 +      sources = recoveryInfo.getSourceDnInfos();
 +      stripedReaders = new ArrayList<>(sources.length);
 +
 +      Preconditions.checkArgument(liveIndices.length >= minRequiredSources,
 +          "No enough live striped blocks.");
 +      Preconditions.checkArgument(liveIndices.length == sources.length,
 +          "liveBlockIndices and source dns should match");
 +
 +      if (minRequiredSources < dataBlkNum) {
 +        zeroStripeBuffers = 
 +            new ByteBuffer[dataBlkNum - minRequiredSources];
 +        zeroStripeIndices = new short[dataBlkNum - minRequiredSources];
 +      }
 +
 +      targets = recoveryInfo.getTargetDnInfos();
 +      targetStorageTypes = recoveryInfo.getTargetStorageTypes();
 +      targetIndices = new short[targets.length];
 +      targetBuffers = new ByteBuffer[targets.length];
 +
 +      Preconditions.checkArgument(targetIndices.length <= parityBlkNum,
 +          "Too much missed striped blocks.");
 +
 +      targetSockets = new Socket[targets.length];
 +      targetOutputStreams = new DataOutputStream[targets.length];
 +      targetInputStreams = new DataInputStream[targets.length];
 +
 +      blockOffset4Targets = new long[targets.length];
 +      seqNo4Targets = new long[targets.length];
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        blockOffset4Targets[i] = 0;
 +        seqNo4Targets[i] = 0;
 +      }
 +
 +      getTargetIndices();
 +      cachingStrategy = CachingStrategy.newDefaultStrategy();
 +    }
 +
 +    private ByteBuffer allocateBuffer(int length) {
 +      return ByteBuffer.allocate(length);
 +    }
 +
 +    private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) {
 +      return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize,
 +          dataBlkNum, i);
 +    }
 +
 +    private long getBlockLen(ExtendedBlock blockGroup, int i) { 
 +      return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(),
 +          cellSize, dataBlkNum, i);
 +    }
 +
 +    /**
 +     * StripedReader is used to read from one source DN, it contains a block
 +     * reader, buffer and striped block index.
 +     * Only allocate StripedReader once for one source, and the StripedReader
 +     * has the same array order with sources. Typically we only need to allocate
 +     * minimum number (minRequiredSources) of StripedReader, and allocate
 +     * new for new source DN if some existing DN invalid or slow.
 +     * If some source DN is corrupt, set the corresponding blockReader to 
 +     * null and will never read from it again.
 +     *  
 +     * @param i the array index of sources
 +     * @param offsetInBlock offset for the internal block
 +     * @return StripedReader
 +     */
 +    private StripedReader addStripedReader(int i, long offsetInBlock) {
 +      StripedReader reader = new StripedReader(liveIndices[i]);
 +      stripedReaders.add(reader);
 +
 +      BlockReader blockReader = newBlockReader(
 +          getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]);
 +      if (blockReader != null) {
 +        initChecksumAndBufferSizeIfNeeded(blockReader);
 +        reader.blockReader = blockReader;
 +      }
 +      reader.buffer = allocateBuffer(bufferSize);
 +      return reader;
 +    }
 +
 +    @Override
 +    public void run() {
 +      datanode.incrementXmitsInProgress();
 +      try {
 +        // Store the array indices of source DNs we have read successfully.
 +        // In each iteration of read, the success list may be updated if
 +        // some source DN is corrupted or slow. And use the updated success
 +        // list of DNs for next iteration read.
 +        int[] success = new int[minRequiredSources];
 +
 +        int nsuccess = 0;
 +        for (int i = 0; 
 +            i < sources.length && nsuccess < minRequiredSources; i++) {
 +          StripedReader reader = addStripedReader(i, 0);
 +          if (reader.blockReader != null) {
 +            success[nsuccess++] = i;
 +          }
 +        }
 +
 +        if (nsuccess < minRequiredSources) {
 +          String error = "Can't find minimum sources required by "
 +              + "recovery, block id: " + blockGroup.getBlockId();
 +          throw new IOException(error);
 +        }
 +
 +        if (zeroStripeBuffers != null) {
 +          for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +            zeroStripeBuffers[i] = allocateBuffer(bufferSize);
 +          }
 +        }
 +
 +        for (int i = 0; i < targets.length; i++) {
 +          targetBuffers[i] = allocateBuffer(bufferSize);
 +        }
 +
 +        checksumSize = checksum.getChecksumSize();
 +        int chunkSize = bytesPerChecksum + checksumSize;
 +        maxChunksPerPacket = Math.max(
 +            (WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN)/chunkSize, 1);
 +        int maxPacketSize = chunkSize * maxChunksPerPacket 
 +            + PacketHeader.PKT_MAX_HEADER_LEN;
 +
 +        packetBuf = new byte[maxPacketSize];
 +        checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)];
 +
 +        // targetsStatus store whether some target is success, it will record
 +        // any failed target once, if some target failed (invalid DN or transfer
 +        // failed), will not transfer data to it any more.
 +        boolean[] targetsStatus = new boolean[targets.length];
 +        if (initTargetStreams(targetsStatus) == 0) {
 +          String error = "All targets are failed.";
 +          throw new IOException(error);
 +        }
 +
 +        long firstStripedBlockLength = getBlockLen(blockGroup, 0);
 +        while (positionInBlock < firstStripedBlockLength) {
 +          int toRead = Math.min(
 +              bufferSize, (int)(firstStripedBlockLength - positionInBlock));
 +          // step1: read from minimum source DNs required for reconstruction.
 +          //   The returned success list is the source DNs we do real read from
 +          success = readMinimumStripedData4Recovery(success);
 +
 +          // step2: decode to reconstruct targets
 +          long remaining = firstStripedBlockLength - positionInBlock;
 +          int toRecoverLen = remaining < bufferSize ? 
 +              (int)remaining : bufferSize;
 +          recoverTargets(success, targetsStatus, toRecoverLen);
 +
 +          // step3: transfer data
 +          if (transferData2Targets(targetsStatus) == 0) {
 +            String error = "Transfer failed for all targets.";
 +            throw new IOException(error);
 +          }
 +
 +          clearBuffers();
 +          positionInBlock += toRead;
 +        }
 +
 +        endTargetBlocks(targetsStatus);
 +
 +        // Currently we don't check the acks for packets, this is similar as
 +        // block replication.
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block: " + blockGroup, e);
 +      } finally {
 +        datanode.decrementXmitsInProgress();
 +        // close block readers
 +        for (StripedReader stripedReader : stripedReaders) {
 +          closeBlockReader(stripedReader.blockReader);
 +        }
 +        for (int i = 0; i < targets.length; i++) {
 +          IOUtils.closeStream(targetOutputStreams[i]);
 +          IOUtils.closeStream(targetInputStreams[i]);
 +          IOUtils.closeStream(targetSockets[i]);
 +        }
 +      }
 +    }
 +
 +    // init checksum from block reader
 +    private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) {
 +      if (checksum == null) {
 +        checksum = blockReader.getDataChecksum();
 +        bytesPerChecksum = checksum.getBytesPerChecksum();
 +        // The bufferSize is flat to divide bytesPerChecksum
 +        int readBufferSize = STRIPED_READ_BUFFER_SIZE;
 +        bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum :
 +          readBufferSize - readBufferSize % bytesPerChecksum;
 +      } else {
 +        assert blockReader.getDataChecksum().equals(checksum);
 +      }
 +    }
 +
 +    private void getTargetIndices() {
 +      BitSet bitset = new BitSet(dataBlkNum + parityBlkNum);
 +      for (int i = 0; i < sources.length; i++) {
 +        bitset.set(liveIndices[i]);
 +      }
 +      int m = 0;
 +      int k = 0;
 +      for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
 +        if (!bitset.get(i)) {
 +          if (getBlockLen(blockGroup, i) > 0) {
 +            if (m < targets.length) {
 +              targetIndices[m++] = (short)i;
 +            }
 +          } else {
 +            zeroStripeIndices[k++] = (short)i;
 +          }
 +        }
 +      }
 +    }
 +
 +    private long getReadLength(int index) {
 +      long blockLen = getBlockLen(blockGroup, index);
 +      long remaining = blockLen - positionInBlock;
 +      return remaining > bufferSize ? bufferSize : remaining;
 +    }
 +
 +    /**
 +     * Read from minimum source DNs required for reconstruction in the iteration.
 +     * First try the success list which we think they are the best DNs
 +     * If source DN is corrupt or slow, try to read some other source DN, 
 +     * and will update the success list. 
 +     * 
 +     * Remember the updated success list and return it for following 
 +     * operations and next iteration read.
 +     * 
 +     * @param success the initial success list of source DNs we think best
 +     * @return updated success list of source DNs we do real read
 +     * @throws IOException
 +     */
 +    private int[] readMinimumStripedData4Recovery(final int[] success)
 +        throws IOException {
 +      int nsuccess = 0;
 +      int[] newSuccess = new int[minRequiredSources];
 +      BitSet used = new BitSet(sources.length);
 +      /*
 +       * Read from minimum source DNs required, the success list contains
 +       * source DNs which we think best.
 +       */
 +      for (int i = 0; i < minRequiredSources; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        if (getReadLength(liveIndices[success[i]]) > 0) {
 +          Callable<Void> readCallable = readFromBlock(
 +              reader.blockReader, reader.buffer);
 +          Future<Void> f = readService.submit(readCallable);
 +          futures.put(f, success[i]);
 +        } else {
 +          // If the read length is 0, we don't need to do real read
 +          reader.buffer.position(0);
 +          newSuccess[nsuccess++] = success[i];
 +        }
 +        used.set(success[i]);
 +      }
 +
 +      while (!futures.isEmpty()) {
 +        try {
 +          StripingChunkReadResult result =
 +              StripedBlockUtil.getNextCompletedStripedRead(
 +                  readService, futures, STRIPED_READ_TIMEOUT_MILLIS);
 +          int resultIndex = -1;
 +          if (result.state == StripingChunkReadResult.SUCCESSFUL) {
 +            resultIndex = result.index;
 +          } else if (result.state == StripingChunkReadResult.FAILED) {
 +            // If read failed for some source DN, we should not use it anymore 
 +            // and schedule read from another source DN.
 +            StripedReader failedReader = stripedReaders.get(result.index);
 +            closeBlockReader(failedReader.blockReader);
 +            failedReader.blockReader = null;
 +            resultIndex = scheduleNewRead(used);
 +          } else if (result.state == StripingChunkReadResult.TIMEOUT) {
 +            // If timeout, we also schedule a new read.
 +            resultIndex = scheduleNewRead(used);
 +          }
 +          if (resultIndex >= 0) {
 +            newSuccess[nsuccess++] = resultIndex;
 +            if (nsuccess >= minRequiredSources) {
 +              // cancel remaining reads if we read successfully from minimum
 +              // number of source DNs required by reconstruction.
 +              cancelReads(futures.keySet());
 +              futures.clear();
 +              break;
 +            }
 +          }
 +        } catch (InterruptedException e) {
 +          LOG.info("Read data interrupted.", e);
 +          break;
 +        }
 +      }
 +
 +      if (nsuccess < minRequiredSources) {
 +        String error = "Can't read data from minimum number of sources "
 +            + "required by reconstruction, block id: " + blockGroup.getBlockId();
 +        throw new IOException(error);
 +      }
 +
 +      return newSuccess;
 +    }
 +    
 +    private void paddingBufferToLen(ByteBuffer buffer, int len) {
 +      int toPadding = len - buffer.position();
 +      for (int i = 0; i < toPadding; i++) {
 +        buffer.put((byte) 0);
 +      }
 +    }
 +    
 +    // Initialize decoder
 +    private void initDecoderIfNecessary() {
 +      if (decoder == null) {
 +        decoder = newDecoder(dataBlkNum, parityBlkNum);
 +      }
 +    }
 +
 +    private int[] getErasedIndices(boolean[] targetsStatus) {
 +      int[] result = new int[targets.length];
 +      int m = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          result[m++] = convertIndex4Decode(targetIndices[i], 
 +              dataBlkNum, parityBlkNum);
 +        }
 +      }
 +      return Arrays.copyOf(result, m);
 +    }
 +
 +    private void recoverTargets(int[] success, boolean[] targetsStatus,
 +        int toRecoverLen) {
 +      initDecoderIfNecessary();
 +      ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum];
 +      for (int i = 0; i < success.length; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        ByteBuffer buffer = reader.buffer;
 +        paddingBufferToLen(buffer, toRecoverLen);
 +        inputs[convertIndex4Decode(reader.index, dataBlkNum, parityBlkNum)] = 
 +            (ByteBuffer)buffer.flip();
 +      }
 +      if (success.length < dataBlkNum) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          ByteBuffer buffer = zeroStripeBuffers[i];
 +          paddingBufferToLen(buffer, toRecoverLen);
 +          int index = convertIndex4Decode(zeroStripeIndices[i], dataBlkNum,
 +              parityBlkNum);
 +          inputs[index] = (ByteBuffer)buffer.flip();
 +        }
 +      }
 +      int[] erasedIndices = getErasedIndices(targetsStatus);
 +      ByteBuffer[] outputs = new ByteBuffer[erasedIndices.length];
 +      int m = 0;
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetsStatus[i]) {
 +          outputs[m++] = targetBuffers[i];
 +          outputs[i].limit(toRecoverLen);
 +        }
 +      }
 +      decoder.decode(inputs, erasedIndices, outputs);
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          long blockLen = getBlockLen(blockGroup, targetIndices[i]);
 +          long remaining = blockLen - positionInBlock;
 +          if (remaining < 0) {
 +            targetBuffers[i].limit(0);
 +          } else if (remaining < toRecoverLen) {
 +            targetBuffers[i].limit((int)remaining);
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Schedule a read from some new source DN if some DN is corrupted
 +     * or slow, this is called from the read iteration.
 +     * Initially we may only have <code>minRequiredSources</code> number of 
 +     * StripedReader.
 +     * If the position is at the end of target block, don't need to do 
 +     * real read, and return the array index of source DN, otherwise -1.
 +     * 
 +     * @param used the used source DNs in this iteration.
 +     * @return the array index of source DN if don't need to do real read.
 +     */
 +    private int scheduleNewRead(BitSet used) {
 +      StripedReader reader = null;
 +      // step1: initially we may only have <code>minRequiredSources</code>
 +      // number of StripedReader, and there may be some source DNs we never 
 +      // read before, so will try to create StripedReader for one new source DN
 +      // and try to read from it. If found, go to step 3.
 +      int m = stripedReaders.size();
 +      while (reader == null && m < sources.length) {
 +        reader = addStripedReader(m, positionInBlock);
 +        if (getReadLength(liveIndices[m]) > 0) {
 +          if (reader.blockReader == null) {
 +            reader = null;
 +            m++;
 +          }
 +        } else {
 +          used.set(m);
 +          return m;
 +        }
 +      }
 +
 +      // step2: if there is no new source DN we can use, try to find a source 
 +      // DN we ever read from but because some reason, e.g., slow, it
 +      // is not in the success DN list at the begin of this iteration, so 
 +      // we have not tried it in this iteration. Now we have a chance to 
 +      // revisit it again.
 +      for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
 +        if (!used.get(i)) {
 +          StripedReader r = stripedReaders.get(i);
 +          if (getReadLength(liveIndices[i]) > 0) {
 +            closeBlockReader(r.blockReader);
 +            r.blockReader = newBlockReader(
 +                getBlock(blockGroup, liveIndices[i]), positionInBlock,
 +                sources[i]);
 +            if (r.blockReader != null) {
 +              m = i;
 +              reader = r;
 +            }
 +          } else {
 +            used.set(i);
 +            r.buffer.position(0);
 +            return i;
 +          }
 +        }
 +      }
 +
 +      // step3: schedule if find a correct source DN and need to do real read.
 +      if (reader != null) {
 +        Callable<Void> readCallable = readFromBlock(
 +            reader.blockReader, reader.buffer);
 +        Future<Void> f = readService.submit(readCallable);
 +        futures.put(f, m);
 +        used.set(m);
 +      }
 +
 +      return -1;
 +    }
 +
 +    // cancel all reads.
 +    private void cancelReads(Collection<Future<Void>> futures) {
 +      for (Future<Void> future : futures) {
 +        future.cancel(true);
 +      }
 +    }
 +
 +    private Callable<Void> readFromBlock(final BlockReader reader,
 +        final ByteBuffer buf) {
 +      return new Callable<Void>() {
 +
 +        @Override
 +        public Void call() throws Exception {
 +          try {
 +            actualReadFromBlock(reader, buf);
 +            return null;
 +          } catch (IOException e) {
 +            LOG.info(e.getMessage());
 +            throw e;
 +          }
 +        }
 +
 +      };
 +    }
 +
 +    /**
 +     * Read bytes from block
 +     */
 +    private void actualReadFromBlock(BlockReader reader, ByteBuffer buf)
 +        throws IOException {
 +      int len = buf.remaining();
 +      int n = 0;
 +      while (n < len) {
 +        int nread = reader.read(buf);
 +        if (nread <= 0) {
 +          break;
 +        }
 +        n += nread;
 +      }
 +    }
 +
 +    // close block reader
 +    private void closeBlockReader(BlockReader blockReader) {
 +      try {
 +        if (blockReader != null) {
 +          blockReader.close();
 +        }
 +      } catch (IOException e) {
 +        // ignore
 +      }
 +    }
 +
 +    private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) {
 +      return NetUtils.createSocketAddr(dnInfo.getXferAddr(
 +          datanode.getDnConf().getConnectToDnViaHostname()));
 +    }
 +
 +    private BlockReader newBlockReader(final ExtendedBlock block, 
 +        long offsetInBlock, DatanodeInfo dnInfo) {
 +      if (offsetInBlock >= block.getNumBytes()) {
 +        return null;
 +      }
 +      try {
 +        InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo);
 +        Token<BlockTokenIdentifier> blockToken = datanode.getBlockAccessToken(
 +            block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ));
 +        /*
 +         * This can be further improved if the replica is local, then we can
 +         * read directly from DN and need to check the replica is FINALIZED
 +         * state, notice we should not use short-circuit local read which
 +         * requires config for domain-socket in UNIX or legacy config in Windows.
++         *
++         * TODO: add proper tracer
 +         */
 +        return RemoteBlockReader2.newBlockReader(
 +            "dummy", block, blockToken, offsetInBlock, 
 +            block.getNumBytes() - offsetInBlock, true,
 +            "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo,
-             null, cachingStrategy);
++            null, cachingStrategy, null);
 +      } catch (IOException e) {
 +        return null;
 +      }
 +    }
 +
 +    private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr,
 +        Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
 +        throws IOException {
 +      Peer peer = null;
 +      boolean success = false;
 +      Socket sock = null;
 +      final int socketTimeout = datanode.getDnConf().getSocketTimeout(); 
 +      try {
 +        sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
 +        NetUtils.connect(sock, addr, socketTimeout);
 +        peer = DFSUtilClient.peerFromSocketAndKey(datanode.getSaslClient(),
 +            sock, datanode.getDataEncryptionKeyFactoryForBlock(b),
 +            blockToken, datanodeId);
 +        peer.setReadTimeout(socketTimeout);
 +        success = true;
 +        return peer;
 +      } finally {
 +        if (!success) {
 +          IOUtils.cleanup(LOG, peer);
 +          IOUtils.closeSocket(sock);
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Send data to targets
 +     */
 +    private int transferData2Targets(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          boolean success = false;
 +          try {
 +            ByteBuffer buffer = targetBuffers[i];
 +            
 +            if (buffer.remaining() == 0) {
 +              continue;
 +            }
 +
 +            checksum.calculateChunkedSums(
 +                buffer.array(), 0, buffer.remaining(), checksumBuf, 0);
 +
 +            int ckOff = 0;
 +            while (buffer.remaining() > 0) {
 +              DFSPacket packet = new DFSPacket(packetBuf, maxChunksPerPacket,
 +                  blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, false);
 +              int maxBytesToPacket = maxChunksPerPacket * bytesPerChecksum;
 +              int toWrite = buffer.remaining() > maxBytesToPacket ?
 +                  maxBytesToPacket : buffer.remaining();
 +              int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * checksumSize;
 +              packet.writeChecksum(checksumBuf, ckOff, ckLen);
 +              ckOff += ckLen;
 +              packet.writeData(buffer, toWrite);
 +
 +              // Send packet
 +              packet.writeTo(targetOutputStreams[i]);
 +
 +              blockOffset4Targets[i] += toWrite;
 +              nsuccess++;
 +              success = true;
 +            }
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +          targetsStatus[i] = success;
 +        }
 +      }
 +      return nsuccess;
 +    }
 +
 +    /**
 +     * clear all buffers
 +     */
 +    private void clearBuffers() {
 +      for (StripedReader stripedReader : stripedReaders) {
 +        if (stripedReader.buffer != null) {
 +          stripedReader.buffer.clear();
 +        }
 +      }
 +
 +      if (zeroStripeBuffers != null) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          zeroStripeBuffers[i].clear();
 +        }
 +      }
 +
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetBuffers[i] != null) {
 +          cleanBuffer(targetBuffers[i]);
 +        }
 +      }
 +    }
 +    
 +    private ByteBuffer cleanBuffer(ByteBuffer buffer) {
 +      Arrays.fill(buffer.array(), (byte) 0);
 +      return (ByteBuffer)buffer.clear();
 +    }
 +
 +    // send an empty packet to mark the end of the block
 +    private void endTargetBlocks(boolean[] targetsStatus) {
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          try {
 +            DFSPacket packet = new DFSPacket(packetBuf, 0, 
 +                blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, true);
 +            packet.writeTo(targetOutputStreams[i]);
 +            targetOutputStreams[i].flush();
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Initialize  output/input streams for transferring data to target
 +     * and send create block request. 
 +     */
 +    private int initTargetStreams(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        Socket socket = null;
 +        DataOutputStream out = null;
 +        DataInputStream in = null;
 +        boolean success = false;
 +        try {
 +          InetSocketAddress targetAddr = 
 +              getSocketAddress4Transfer(targets[i]);
 +          socket = datanode.newSocket();
 +          NetUtils.connect(socket, targetAddr, 
 +              datanode.getDnConf().getSocketTimeout());
 +          socket.setSoTimeout(datanode.getDnConf().getSocketTimeout());
 +
 +          ExtendedBlock block = getBlock(blockGroup, targetIndices[i]);
 +          Token<BlockTokenIdentifier> blockToken = 
 +              datanode.getBlockAccessToken(block,
 +                  EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 +
 +          long writeTimeout = datanode.getDnConf().getSocketWriteTimeout();
 +          OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout);
 +          InputStream unbufIn = NetUtils.getInputStream(socket);
 +          DataEncryptionKeyFactory keyFactory =
 +            datanode.getDataEncryptionKeyFactoryForBlock(block);
 +          IOStreamPair saslStreams = datanode.getSaslClient().socketSend(
 +              socket, unbufOut, unbufIn, keyFactory, blockToken, targets[i]);
 +
 +          unbufOut = saslStreams.out;
 +          unbufIn = saslStreams.in;
 +
 +          out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-               DFSUtil.getSmallBufferSize(conf)));
++              DFSUtilClient.getSmallBufferSize(conf)));
 +          in = new DataInputStream(unbufIn);
 +
 +          DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId());
 +          new Sender(out).writeBlock(block, targetStorageTypes[i], 
 +              blockToken, "", new DatanodeInfo[]{targets[i]}, 
 +              new StorageType[]{targetStorageTypes[i]}, source, 
 +              BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0, 
 +              checksum, cachingStrategy, false, false, null);
 +
 +          targetSockets[i] = socket;
 +          targetOutputStreams[i] = out;
 +          targetInputStreams[i] = in;
 +          nsuccess++;
 +          success = true;
 +        } catch (Throwable e) {
 +          LOG.warn(e.getMessage());
 +        } finally {
 +          if (!success) {
 +            IOUtils.closeStream(out);
 +            IOUtils.closeStream(in);
 +            IOUtils.closeStream(socket);
 +          }
 +        }
 +        targetsStatus[i] = success;
 +      }
 +      return nsuccess;
 +    }
 +  }
 +
 +  private static class StripedReader {
 +    private final short index; // internal block index
 +    private BlockReader blockReader;
 +    private ByteBuffer buffer;
 +
 +    private StripedReader(short index) {
 +      this.index = index;
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 3e001d3,98af592..67c6fc1
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@@ -180,9 -178,8 +180,10 @@@ class FSDirStatAndListingOp 
        }
  
        final FileEncryptionInfo feInfo = isReservedName ? null
-           : fsd.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
+           : FSDirEncryptionZoneOp.getFileEncryptionInfo(fsd, inode,
 -              iip.getPathSnapshotId(), iip);
++          iip.getPathSnapshotId(), iip);
 +      final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.
 +          getErasureCodingPolicy(fsd.getFSNamesystem(), iip);
  
        final LocatedBlocks blocks = bm.createLocatedBlocks(
            inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
@@@ -443,12 -440,9 +444,12 @@@
      long blocksize = 0;
      final boolean isEncrypted;
  
-     final FileEncryptionInfo feInfo = isRawPath ? null :
-         fsd.getFileEncryptionInfo(node, snapshot, iip);
+     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
+         .getFileEncryptionInfo(fsd, node, snapshot, iip);
  
 +    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
 +        fsd.getFSNamesystem(), iip);
 +
      if (node.isFile()) {
        final INodeFile fileNode = node.asFile();
        size = fileNode.computeFileSize(snapshot);
@@@ -500,10 -495,8 +503,10 @@@
      long blocksize = 0;
      LocatedBlocks loc = null;
      final boolean isEncrypted;
-     final FileEncryptionInfo feInfo = isRawPath ? null :
-         fsd.getFileEncryptionInfo(node, snapshot, iip);
+     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
+         .getFileEncryptionInfo(fsd, node, snapshot, iip);
 +    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
 +        fsd.getFSNamesystem(), iip);
      if (node.isFile()) {
        final INodeFile fileNode = node.asFile();
        size = fileNode.computeFileSize(snapshot);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index b6b151c,4dda27d..a94b61c
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@@ -131,11 -131,9 +131,10 @@@ import org.apache.commons.logging.impl.
  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.CryptoProtocolVersion;
- import org.apache.hadoop.crypto.key.KeyProvider;
+ import org.apache.hadoop.crypto.key.KeyProvider.Metadata;
  import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 +import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
  import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
  import org.apache.hadoop.fs.CacheFlag;
  import org.apache.hadoop.fs.ContentSummary;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 4143964,79a3773..e9363b4
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@@ -2057,18 -2043,6 +2061,18 @@@ class NameNodeRpcServer implements Name
    public void removeSpanReceiver(long id) throws IOException {
      checkNNStartup();
      namesystem.checkSuperuserPrivilege();
-     nn.spanReceiverHost.removeSpanReceiver(id);
+     nn.tracerConfigurationManager.removeSpanReceiver(id);
    }
 +
 +  @Override // ClientProtocol
 +  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
 +    checkNNStartup();
 +    return namesystem.getErasureCodingPolicies();
 +  }
 +
 +  @Override // ClientProtocol
 +  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
 +    checkNNStartup();
 +    return namesystem.getErasureCodingPolicy(src);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 9d24b91,727259f..6dd7b89
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@@ -33,7 -33,7 +33,8 @@@ package hadoop.hdfs.datanode
  
  import "HAServiceProtocol.proto";
  import "hdfs.proto";
 +import "erasurecoding.proto";
+ import "HdfsServer.proto";
  
  /**
   * Information to identify a datanode to a namenode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
index 0000000,3b60e51..66b2a33
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
@@@ -1,0 -1,201 +1,198 @@@
+ /**
+  * 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.
+  */
+ 
+ /**
+  * These .proto interfaces are private and stable.
+  * Please see http://wiki.apache.org/hadoop/Compatibility
+  * for what changes are allowed for a *stable* .proto interface.
+  */
+ 
+ // This file contains protocol buffers that are used throughout HDFS -- i.e.
+ // by the client, server, and data transfer protocols.
+ 
+ 
+ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+ option java_outer_classname = "HdfsServerProtos";
+ option java_generate_equals_and_hash = true;
+ package hadoop.hdfs;
+ 
+ import "hdfs.proto";
+ 
+ /**
 - * A list of storage IDs.
 - */
 -message StorageUuidsProto {
 -  repeated string storageUuids = 1;
 -}
 -
 -/**
+  * Block access token information
+  */
+ message BlockKeyProto {
+   required uint32 keyId = 1;      // Key identifier
+   required uint64 expiryDate = 2; // Expiry time in milliseconds
+   optional bytes keyBytes = 3;    // Key secret
+ }
+ 
+ /**
+  * Current key and set of block keys at the namenode.
+  */
+ message ExportedBlockKeysProto {
+   required bool isBlockTokenEnabled = 1;
+   required uint64 keyUpdateInterval = 2;
+   required uint64 tokenLifeTime = 3;
+   required BlockKeyProto currentKey = 4;
+   repeated BlockKeyProto allKeys = 5;
+ }
+ 
+ /**
+  * Block and datanodes where is it located
+  */
+ message BlockWithLocationsProto {
+   required BlockProto block = 1;   // Block
+   repeated string datanodeUuids = 2; // Datanodes with replicas of the block
+   repeated string storageUuids = 3;  // Storages with replicas of the block
+   repeated StorageTypeProto storageTypes = 4;
++
++  optional bytes indices = 5;
++  optional uint32 dataBlockNum = 6;
++  optional uint32 cellSize = 7;
+ }
+ 
+ /**
+  * List of block with locations
+  */
+ message BlocksWithLocationsProto {
+   repeated BlockWithLocationsProto blocks = 1;
+ }
+ 
+ /**
+  * Editlog information with available transactions
+  */
+ message RemoteEditLogProto {
+   required uint64 startTxId = 1;  // Starting available edit log transaction
+   required uint64 endTxId = 2;    // Ending available edit log transaction
+   optional bool isInProgress = 3 [default = false];
+ }
+ 
+ /**
+  * Enumeration of editlogs available on a remote namenode
+  */
+ message RemoteEditLogManifestProto {
+   repeated RemoteEditLogProto logs = 1;
+ }
+ 
+ /**
+  * Namespace information that describes namespace on a namenode
+  */
+ message NamespaceInfoProto {
+   required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
+   required uint32 unused = 2;               // Retained for backward compatibility
+   required string blockPoolID = 3;          // block pool used by the namespace
+   required StorageInfoProto storageInfo = 4;// Node information
+   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+   optional uint64 capabilities = 6 [default = 0]; // feature flags
+ }
+ 
+ /**
+  * State of a block replica at a datanode
+  */
+ enum ReplicaStateProto {
+   FINALIZED = 0;  // State of a replica when it is not modified
+   RBW = 1;        // State of replica that is being written to
+   RWR = 2;        // State of replica that is waiting to be recovered
+   RUR = 3;        // State of replica that is under recovery
+   TEMPORARY = 4;  // State of replica that is created for replication
+ }
+ 
+ /**
+  * Block that needs to be recovered with at a given location
+  */
+ message RecoveringBlockProto {
+   required uint64 newGenStamp = 1;        // New genstamp post recovery
+   required LocatedBlockProto block = 2;   // Block to be recovered
+   optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
+ }
+ 
+ /**
+  * Unique signature to identify checkpoint transactions.
+  */
+ message CheckpointSignatureProto {
+   required string blockPoolId = 1;
+   required uint64 mostRecentCheckpointTxId = 2;
+   required uint64 curSegmentTxId = 3;
+   required StorageInfoProto storageInfo = 4;
+ }
+ 
+ /**
+  * Command returned from primary to checkpointing namenode.
+  * This command has checkpoint signature that identifies
+  * checkpoint transaction and is needed for further
+  * communication related to checkpointing.
+  */
+ message CheckpointCommandProto {
+   // Unique signature to identify checkpoint transation
+   required CheckpointSignatureProto signature = 1;
+ 
+   // If true, return transfer image to primary upon the completion of checkpoint
+   required bool needToReturnImage = 2;
+ }
+ 
+ /**
+  * Command sent from one namenode to another namenode.
+  */
+ message NamenodeCommandProto {
+   enum Type {
+     NamenodeCommand = 0;      // Base command
+     CheckPointCommand = 1;    // Check point command
+   }
+   required uint32 action = 1;
+   required Type type = 2;
+   optional CheckpointCommandProto checkpointCmd = 3;
+ }
+ 
+ /**
+  * void request
+  */
+ message VersionRequestProto {
+ }
+ 
+ /**
+  * Version response from namenode.
+  */
+ message VersionResponseProto {
+   required NamespaceInfoProto info = 1;
+ }
+ 
+ /**
+  * Common node information shared by all the nodes in the cluster
+  */
+ message StorageInfoProto {
+   required uint32 layoutVersion = 1; // Layout version of the file system
+   required uint32 namespceID = 2;    // File system namespace ID
+   required string clusterID = 3;     // ID of the cluster
+   required uint64 cTime = 4;         // File system creation time
+ }
+ 
+ /**
+  * Information sent by a namenode to identify itself to the primary namenode.
+  */
+ message NamenodeRegistrationProto {
+   required string rpcAddress = 1;    // host:port of the namenode RPC address
+   required string httpAddress = 2;   // host:port of the namenode http server
+   enum NamenodeRoleProto {
+     NAMENODE = 1;
+     BACKUP = 2;
+     CHECKPOINT = 3;
+   }
+   required StorageInfoProto storageInfo = 3;  // Node information
+   optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index d1b16b1,50d548a..ce7aee3
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@@ -73,21 -74,17 +73,21 @@@ public class TestBlockTokenWithDFS 
    private static final String FILE_TO_READ = "/fileToRead.dat";
    private static final String FILE_TO_WRITE = "/fileToWrite.dat";
    private static final String FILE_TO_APPEND = "/fileToAppend.dat";
 -  private final byte[] rawData = new byte[FILE_SIZE];
  
    {
-     ((Log4JLogger) DFSClient.LOG).getLogger().setLevel(Level.ALL);
+     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
 +  }
 +
 +  public static byte[] generateBytes(int fileSize){
      Random r = new Random();
 +    byte[] rawData = new byte[fileSize];
      r.nextBytes(rawData);
 +    return rawData;
    }
  
 -  private void createFile(FileSystem fs, Path filename) throws IOException {
 +  private void createFile(FileSystem fs, Path filename, byte[] expected) throws IOException {
      FSDataOutputStream out = fs.create(filename);
 -    out.write(rawData);
 +    out.write(expected);
      out.close();
    }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 917b0f2,df07a62..2bb3d5f
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@@ -55,9 -46,8 +54,10 @@@ import org.apache.hadoop.hdfs.server.bl
  import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
  import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
  import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 +import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.io.IOUtils;
+ import org.apache.hadoop.test.GenericTestUtils;
  import org.apache.hadoop.test.PathUtils;
  import org.apache.log4j.Level;
  import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 5efc94d,a84ddd0..6df88fd
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@@ -1660,60 -1657,4 +1662,60 @@@ public class TestFsck 
        }
      }
    }
 +
 +  @Test
 +  public void testECFsck() throws Exception {
 +    MiniDFSCluster cluster = null;
 +    FileSystem fs = null;
 +    try {
 +      Configuration conf = new HdfsConfiguration();
 +      final long precision = 1L;
 +      conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
 +      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
 +      int totalSize = ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumDataUnits()
 +                      + ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumParityUnits();
 +      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build();
 +      fs = cluster.getFileSystem();
 +
 +      // create a contiguous file
 +      Path replDirPath = new Path("/replicated");
 +      Path replFilePath = new Path(replDirPath, "replfile");
 +      final short factor = 3;
 +      DFSTestUtil.createFile(fs, replFilePath, 1024, factor, 0);
 +      DFSTestUtil.waitReplication(fs, replFilePath, factor);
 +
 +      // create a large striped file
 +      Path ecDirPath = new Path("/striped");
 +      Path largeFilePath = new Path(ecDirPath, "largeFile");
 +      DFSTestUtil.createStripedFile(cluster, largeFilePath, ecDirPath, 1, 2, true);
 +
 +      // create a small striped file
 +      Path smallFilePath = new Path(ecDirPath, "smallFile");
 +      DFSTestUtil.writeFile(fs, smallFilePath, "hello world!");
 +
 +      long replTime = fs.getFileStatus(replFilePath).getAccessTime();
 +      long ecTime = fs.getFileStatus(largeFilePath).getAccessTime();
 +      Thread.sleep(precision);
 +      setupAuditLogs();
 +      String outStr = runFsck(conf, 0, true, "/");
 +      verifyAuditLogs();
 +      assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime());
 +      assertEquals(ecTime, fs.getFileStatus(largeFilePath).getAccessTime());
 +      System.out.println(outStr);
 +      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
 +      if (fs != null) {try{fs.close();} catch(Exception e){}}
 +      cluster.shutdown();
 +
 +      // restart the cluster; bring up namenode but not the data nodes
 +      cluster = new MiniDFSCluster.Builder(conf)
 +          .numDataNodes(0).format(false).build();
 +      outStr = runFsck(conf, 1, true, "/");
 +      // expect the result is corrupt
 +      assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
 +      System.out.println(outStr);
 +    } finally {
 +      if (fs != null) {try{fs.close();} catch(Exception e){}}
 +      if (cluster != null) { cluster.shutdown(); }
 +    }
 +  }
- }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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();
+  }
+}


[21/58] [abbrv] hadoop git commit: YARN-4204. ConcurrentModificationException in FairSchedulerQueueInfo. (adhoot)

Posted by zh...@apache.org.
YARN-4204. ConcurrentModificationException in FairSchedulerQueueInfo. (adhoot)


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

Branch: refs/heads/HDFS-7285
Commit: fb2e525c0775ccf218c8980676e9fb4005a406a6
Parents: 892ade6
Author: Anubhav Dhoot <ad...@apache.org>
Authored: Sun Sep 27 20:52:38 2015 -0700
Committer: Anubhav Dhoot <ad...@apache.org>
Committed: Mon Sep 28 09:05:45 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 +
 .../scheduler/fair/FSLeafQueue.java             |  5 +-
 .../scheduler/fair/FSParentQueue.java           |  3 +-
 .../scheduler/fair/QueueManager.java            | 24 +++---
 .../scheduler/fair/TestFSParentQueue.java       | 79 ++++++++++++++++++++
 5 files changed, 100 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb2e525c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f7ea26e..54207aa 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -902,6 +902,8 @@ Release 2.8.0 - UNRELEASED
     YARN-4044. Running applications information changes such as movequeue is not published to 
     TimeLine server. (Sunil G via rohithsharmaks)
 
+    YARN-4204. ConcurrentModificationException in FairSchedulerQueueInfo. (adhoot)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb2e525c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index f90a198..ca5a146 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -70,7 +70,8 @@ public class FSLeafQueue extends FSQueue {
   private Resource amResourceUsage;
 
   private final ActiveUsersManager activeUsersManager;
-  
+  public static final List<FSQueue> EMPTY_LIST = Collections.emptyList();
+
   public FSLeafQueue(String name, FairScheduler scheduler,
       FSParentQueue parent) {
     super(name, scheduler, parent);
@@ -383,7 +384,7 @@ public class FSLeafQueue extends FSQueue {
 
   @Override
   public List<FSQueue> getChildQueues() {
-    return new ArrayList<FSQueue>(1);
+    return EMPTY_LIST;
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb2e525c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
index 7d2e5b8..febe050 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
@@ -27,6 +27,7 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -279,7 +280,7 @@ public class FSParentQueue extends FSQueue {
   public List<FSQueue> getChildQueues() {
     readLock.lock();
     try {
-      return Collections.unmodifiableList(childQueues);
+      return ImmutableList.copyOf(childQueues);
     } finally {
       readLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb2e525c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
index 6556717..0092845 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
@@ -28,6 +28,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 
 import javax.xml.parsers.ParserConfigurationException;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -295,17 +296,18 @@ public class QueueManager {
    * Remove a queue and all its descendents.
    */
   private void removeQueue(FSQueue queue) {
-    if (queue instanceof FSLeafQueue) {
-      leafQueues.remove(queue);
-    } else {
-      List<FSQueue> childQueues = queue.getChildQueues();
-      while (!childQueues.isEmpty()) {
-        removeQueue(childQueues.get(0));
+    synchronized (queues) {
+      if (queue instanceof FSLeafQueue) {
+        leafQueues.remove(queue);
+      } else {
+        for (FSQueue childQueue:queue.getChildQueues()) {
+          removeQueue(childQueue);
+        }
       }
+      queues.remove(queue.getName());
+      FSParentQueue parent = queue.getParent();
+      parent.removeChildQueue(queue);
     }
-    queues.remove(queue.getName());
-    FSParentQueue parent = queue.getParent();
-    parent.removeChildQueue(queue);
   }
   
   /**
@@ -360,7 +362,9 @@ public class QueueManager {
    * Get a collection of all queues
    */
   public Collection<FSQueue> getQueues() {
-    return queues.values();
+    synchronized (queues) {
+      return ImmutableList.copyOf(queues.values());
+    }
   }
   
   private String ensureRootPrefix(String name) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb2e525c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSParentQueue.java
new file mode 100644
index 0000000..f3e9e0c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSParentQueue.java
@@ -0,0 +1,79 @@
+/**
+ * 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.yarn.server.resourcemanager.scheduler.fair;
+
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestFSParentQueue {
+
+  private FairSchedulerConfiguration conf;
+  private QueueManager queueManager;
+  private Set<FSQueue> notEmptyQueues;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new FairSchedulerConfiguration();
+    FairScheduler scheduler = mock(FairScheduler.class);
+    AllocationConfiguration allocConf = new AllocationConfiguration(conf);
+    when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
+    when(scheduler.getConf()).thenReturn(conf);
+    SystemClock clock = new SystemClock();
+    when(scheduler.getClock()).thenReturn(clock);
+    notEmptyQueues = new HashSet<FSQueue>();
+    queueManager = new QueueManager(scheduler) {
+      @Override
+      public boolean isEmpty(FSQueue queue) {
+        return !notEmptyQueues.contains(queue);
+      }
+    };
+    FSQueueMetrics.forQueue("root", null, true, conf);
+    queueManager.initialize(conf);
+  }
+
+  @Test
+  public void testConcurrentChangeToGetChildQueue() {
+
+    queueManager.getLeafQueue("parent.child", true);
+    queueManager.getLeafQueue("parent.child2", true);
+    FSParentQueue test = queueManager.getParentQueue("parent", false);
+    assertEquals(2, test.getChildQueues().size());
+
+    boolean first = true;
+    int childQueuesFound = 0;
+    for (FSQueue childQueue:test.getChildQueues()) {
+      if (first) {
+        first = false;
+        queueManager.getLeafQueue("parent.child3", true);
+      }
+      childQueuesFound++;
+    }
+
+    assertEquals(2, childQueuesFound);
+    assertEquals(3, test.getChildQueues().size());
+  }
+}


[25/58] [abbrv] hadoop git commit: MAPREDUCE-6492. AsyncDispatcher exit with NPE on TaskAttemptImpl#sendJHStartEventForAssignedFailTask. Contributed by Bibin A Chundatt

Posted by zh...@apache.org.
MAPREDUCE-6492. AsyncDispatcher exit with NPE on TaskAttemptImpl#sendJHStartEventForAssignedFailTask. Contributed by Bibin A Chundatt


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

Branch: refs/heads/HDFS-7285
Commit: ab11085b81353e1617875deb10f3c8e2a8b91a1e
Parents: e5992ef
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Sep 28 22:12:03 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Sep 28 22:12:03 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                         | 4 ++++
 .../hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java    | 4 +++-
 .../hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java    | 8 ++++++++
 3 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab11085b/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 67adcbd..da5ee0b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -626,6 +626,10 @@ Release 2.7.2 - UNRELEASED
     MAPREDUCE-5982. Task attempts that fail from the ASSIGNED state can
     disappear (Chang Li via jlowe)
 
+    MAPREDUCE-6492. AsyncDispatcher exit with NPE on
+    TaskAttemptImpl#sendJHStartEventForAssignedFailTask (Bibin A Chundatt via
+    jlowe)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab11085b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index a7becdb..db4f585 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -1486,7 +1486,9 @@ public abstract class TaskAttemptImpl implements
 
   private static void
       sendJHStartEventForAssignedFailTask(TaskAttemptImpl taskAttempt) {
-    TaskAttemptContainerLaunchedEvent event;
+    if (null == taskAttempt.container) {
+      return;
+    }
     taskAttempt.launchTime = taskAttempt.clock.getTime();
 
     InetSocketAddress nodeHttpInetAddr =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab11085b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
index 6b4656a..1ed8098 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
@@ -774,6 +774,14 @@ public class TestTaskAttempt{
     assertFalse(
         "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task",
         eventHandler.internalError);
+    try {
+      taImpl.handle(new TaskAttemptEvent(attemptId,
+          TaskAttemptEventType.TA_KILL));
+      Assert.assertTrue("No exception on UNASSIGNED STATE KILL event", true);
+    } catch (Exception e) {
+      Assert.assertFalse(
+          "Exception not expected for UNASSIGNED STATE KILL event", true);
+    }
   }
 
   @Test


[45/58] [abbrv] hadoop git commit: Add HADOOP-12447 to CHANGES.txt

Posted by zh...@apache.org.
Add HADOOP-12447 to CHANGES.txt


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

Branch: refs/heads/HDFS-7285
Commit: a2efb945e31700ae9a5afecf9f24294cff39a14f
Parents: 850d679
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Sep 29 09:37:13 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Sep 29 09:37:13 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2efb945/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2af6580..ec7d1c6 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1098,6 +1098,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12440. TestRPC#testRPCServerShutdown did not produce the desired
     thread states before shutting down. (Xiao Chen via mingma)
 
+    HADOOP-12447. Clean up some htrace integration issues (cmccabe)
+
   OPTIMIZATIONS
 
     HADOOP-12051. ProtobufRpcEngine.invoke() should use Exception.toString()


[40/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 0000000,92d117c..be346a4
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,3135 +1,3240 @@@
+ /**
+  * 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.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.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.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.FsTracer;
+ 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.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.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.hdfs.protocol.ErasureCodingPolicy;
+ import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
+ import org.apache.hadoop.ipc.RPC;
+ import org.apache.hadoop.ipc.RemoteException;
+ 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.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.core.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.apache.htrace.core.Tracer;
+ 
+ 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 Tracer tracer;
+   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 static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
+   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 {
+     // Copy only the required DFSClient configuration
+     this.tracer = FsTracer.get(conf);
+     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.initThreadsNumForHedgedReads(dfsClientConf.
++          getHedgedReadThreadpoolSize());
+     }
++
++    this.initThreadsNumForStripedReads(dfsClientConf.
++        getStripedReadThreadpoolSize());
+     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 {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("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 {
+     checkOpen();
+     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 = tracer.newScope("getDelegationToken");
+     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 {
+     checkOpen();
+     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 = newPathTraceScope("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 = newPathTraceScope("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 {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("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();
+     }
+   }
+   
+   /**
+    * Decrypts a EDEK by consulting the KeyProvider.
+    */
+   private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
+       feInfo) throws IOException {
+     TraceScope scope = tracer.newScope("decryptEDEK");
+     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 = newPathTraceScope("newDFSInputStream", src);
+     try {
 -      return new DFSInputStream(this, src, verifyChecksum, null);
++      LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0);
++      if (locatedBlocks != null) {
++        ErasureCodingPolicy ecPolicy = locatedBlocks.getErasureCodingPolicy();
++        if (ecPolicy != null) {
++          return new DFSStripedInputStream(this, src, verifyChecksum, ecPolicy,
++              locatedBlocks);
++        }
++        return new DFSInputStream(this, src, verifyChecksum, locatedBlocks);
++      } else {
++        throw new IOException("Cannot open filename " + src);
++      }
+     } 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, 
++    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 {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("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 = newPathTraceScope("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 {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("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 {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("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 = newPathTraceScope("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 {
+     checkOpen();
+     TraceScope scope = tracer.newScope("getStoragePolicies");
+     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 = newSrcDstTraceScope("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 = tracer.newScope("concat");
+     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 = newSrcDstTraceScope("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 + ".");
+     }
+     TraceScope scope = newPathTraceScope("truncate", src);
+     try {
+       return namenode.truncate(src, newLength, clientName);
+     } catch (RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+           UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * 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 = newPathTraceScope("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 = newPathTraceScope("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 = newPathTraceScope("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 = newPathTraceScope("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 = newPathTraceScope("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 = newPathTraceScope("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 = newPathTraceScope("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 = tracer.newScope("getStats");
+     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 = newPathTraceScope("listCorruptFileBlocks", path);
+     try {
+       return namenode.listCorruptFileBlocks(path, cookie);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   public DatanodeInfo[] datanodeReport(DatanodeReportType type)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("datanodeReport");
+     try {
+       return namenode.getDatanodeReport(type);
+     } finally {
+       scope.close();
+     }
+   }
+     
+   public DatanodeStorageReport[] getDatanodeStorageReport(
+       DatanodeReportType type) throws IOException {
+     checkOpen();
+     TraceScope scope =
+         tracer.newScope("datanodeStorageReport");
+     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 {
+     checkOpen();
+     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 =
+         tracer.newScope("setSafeMode");
+     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 = tracer.newScope("createSnapshot");
+     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 = tracer.newScope("deleteSnapshot");
+     try {
+       namenode.deleteSnapshot(snapshotRoot, snapshotName);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException();
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Rename a snapshot.
+    * @param snapshotDir The directory path where the snapshot was taken
+    * @param snapshotOldName Old name of the snapshot
+    * @param snapshotNewName New name of the snapshot
+    * @throws IOException
+    * @see ClientProtocol#renameSnapshot(String, String, String)
+    */
+   public void renameSnapshot(String snapshotDir, String snapshotOldName,
+       String snapshotNewName) throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("renameSnapshot");
+     try {
+       namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException();
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Get all the current snapshottable directories.
+    * @return All the c

<TRUNCATED>

[48/58] [abbrv] hadoop git commit: HDFS-8696. Make the lower and higher watermark in the DN Netty server configurable. Contributed by Xiaobing Zhou.

Posted by zh...@apache.org.
HDFS-8696. Make the lower and higher watermark in the DN Netty server configurable. Contributed by Xiaobing Zhou.


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

Branch: refs/heads/HDFS-7285
Commit: 6f335e4f0ea857ef11ff24c392bf6e549b5db406
Parents: 8703301
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 29 14:59:13 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 29 14:59:13 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt          |  3 +++
 .../java/org/apache/hadoop/hdfs/DFSConfigKeys.java   |  6 ++++++
 .../hdfs/server/datanode/web/DatanodeHttpServer.java | 15 +++++++++++++++
 3 files changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f335e4f/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 0bca0e4..7b62b97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -997,6 +997,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9165. Move entries in META-INF/services/o.a.h.fs.FileSystem to
     hdfs-client. (Mingliang Liu via wheat9)
 
+    HDFS-8696. Make the lower and higher watermark in the DN Netty server
+    configurable. (Xiaobing Zhou via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f335e4f/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 055d7e3..37d74e3 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
@@ -63,6 +63,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       HdfsClientConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT;
   public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
+  public static final String  DFS_WEBHDFS_NETTY_LOW_WATERMARK =
+      "dfs.webhdfs.netty.low.watermark";
+  public static final int  DFS_WEBHDFS_NETTY_LOW_WATERMARK_DEFAULT = 32768;
+  public static final String  DFS_WEBHDFS_NETTY_HIGH_WATERMARK =
+      "dfs.webhdfs.netty.high.watermark";
+  public static final int  DFS_WEBHDFS_NETTY_HIGH_WATERMARK_DEFAULT = 65535;
 
   // HA related configuration
   public static final String  DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f335e4f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
index 62c98e7..441d520 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
@@ -21,6 +21,7 @@ import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.ChannelFactory;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
 import io.netty.channel.ChannelPipeline;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.nio.NioEventLoopGroup;
@@ -30,10 +31,12 @@ import io.netty.handler.codec.http.HttpRequestDecoder;
 import io.netty.handler.codec.http.HttpResponseEncoder;
 import io.netty.handler.ssl.SslHandler;
 import io.netty.handler.stream.ChunkedWriteHandler;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@@ -117,6 +120,18 @@ public class DatanodeHttpServer implements Closeable {
               conf, confForCreate));
         }
       });
+
+      this.httpServer.childOption(
+          ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK,
+          conf.getInt(
+              DFSConfigKeys.DFS_WEBHDFS_NETTY_HIGH_WATERMARK,
+              DFSConfigKeys.DFS_WEBHDFS_NETTY_HIGH_WATERMARK_DEFAULT));
+      this.httpServer.childOption(
+          ChannelOption.WRITE_BUFFER_LOW_WATER_MARK,
+          conf.getInt(
+              DFSConfigKeys.DFS_WEBHDFS_NETTY_LOW_WATERMARK,
+              DFSConfigKeys.DFS_WEBHDFS_NETTY_LOW_WATERMARK_DEFAULT));
+
       if (externalHttpChannel == null) {
         httpServer.channel(NioServerSocketChannel.class);
       } else {


[13/58] [abbrv] hadoop git commit: HDFS-9087. Add some jitter to DataNode.checkDiskErrorThread (Elliott Clark via Colin P. McCabe)

Posted by zh...@apache.org.
HDFS-9087. Add some jitter to DataNode.checkDiskErrorThread (Elliott Clark via Colin P. McCabe)


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

Branch: refs/heads/HDFS-7285
Commit: 0b31c237f2622e256726fc5d7698f0f195dbdbc1
Parents: bf37d3d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Sat Sep 26 20:36:24 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Sat Sep 26 20:36:24 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 .../org/apache/hadoop/hdfs/server/datanode/DataNode.java     | 8 +++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b31c237/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 4ebf437..8c8afed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -980,6 +980,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8053. Move DFSIn/OutputStream and related classes to
     hadoop-hdfs-client. (Mingliang Liu via wheat9)
 
+    HDFS-9087. Add some jitter to DataNode.checkDiskErrorThread (Elliott Clark
+    via Colin P. McCabe) 
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b31c237/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 1bb4485..337706e 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
@@ -88,6 +88,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -356,7 +357,7 @@ public class DataNode extends ReconfigurableBase
   SaslDataTransferServer saslServer;
   private ObjectName dataNodeInfoBeanName;
   private Thread checkDiskErrorThread = null;
-  protected final int checkDiskErrorInterval = 5*1000;
+  protected final int checkDiskErrorInterval;
   private boolean checkDiskErrorFlag = false;
   private Object checkDiskErrorMutex = new Object();
   private long lastDiskErrorCheck;
@@ -387,6 +388,8 @@ public class DataNode extends ReconfigurableBase
     this.connectToDnViaHostname = false;
     this.blockScanner = new BlockScanner(this, conf);
     this.pipelineSupportECN = false;
+    this.checkDiskErrorInterval =
+        ThreadLocalRandom.current().nextInt(5000, (int) (5000 * 1.25));
     initOOBTimeout();
   }
 
@@ -422,6 +425,9 @@ public class DataNode extends ReconfigurableBase
         ",hdfs-" +
         conf.get("hadoop.hdfs.configuration.version", "UNSPECIFIED");
 
+    this.checkDiskErrorInterval =
+        ThreadLocalRandom.current().nextInt(5000, (int) (5000 * 1.25));
+
     // Determine whether we should try to pass file descriptors to clients.
     if (conf.getBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY,
               HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT)) {


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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 5bd48af..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ /dev/null
@@ -1,3141 +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.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.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.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.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.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.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 {
-    checkOpen();
-    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 {
-    checkOpen();
-    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 {
-    checkOpen();
-    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 {
-    checkOpen();
-    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();
-    }
-  }
-  
-  /**
-   * 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 {
-    checkOpen();
-    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 {
-    checkOpen();
-    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 {
-    checkOpen();
-    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 {
-    checkOpen();
-    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 + ".");
-    }
-    TraceScope scope = getPathTraceScope("truncate", src);
-    try {
-      return namenode.truncate(src, newLength, clientName);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-          UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * 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 {
-    checkOpen();
-    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 {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Rename a snapshot.
-   * @param snapshotDir The directory path where the snapshot was taken
-   * @param snapshotOldName Old name of the snapshot
-   * @param snapshotNewName New name of the snapshot
-   * @throws IOException
-   * @see ClientProtocol#renameSnapshot(String, String, String)
-   */
-  public void renameSnapshot(String snapshotDir, String snapshotOldName,
-      String snapshotNewName) throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("renameSnapshot", traceSampler);
-    try {
-      namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Get all the current snapshottable directories.
-   * @return All the current snapshottable directories
-   * @throws IOException
-   * @see ClientProtocol#getSnapshottableDirListing()
-   */
-  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
-      throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("getSnapshottableDirListing",
-        traceSampler);
-    try {
-      return namenode.getSnapshottableDirListing();
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {
-      scope.close();
-    }
-  }
-
-  /**
-   * Allow snapshot on a directory.
-   * 
-   * @see ClientProtocol#allowSnapshot(String snapshotRoot)
-   */
-  public void allowSnapshot(String snapshotRoot) throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("allowSnapshot", traceSampler);
-    try {
-      namenode.allowSnapshot(snapshotRoot);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Disallow snapshot on a directory.
-   * 
-   * @see ClientProtocol#disallowSnapshot(String snapshotRoot)
-   */
-  public void disallowSnapshot(String snapshotRoot) throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("disallowSnapshot", traceSampler);
-    try {
-      namenode.disallowSnapshot(snapshotRoot);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {
-      scope.close();
-    }
-  }
-  
-  /**
-   * Get the difference between two snapshots, or between a snapshot and the
-   * current tree of a directory.
-   * @see ClientProtocol#getSnapshotDiffReport(String, String, String)
-   */
-  public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
-      String fromSnapshot, String toSnapshot) throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("getSnapshotDiffReport", traceSampler);
-    try {
-      return namenode.getSnapshotDiffReport(snapshotDir,
-          fromSnapshot, toSnapshot);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {
-      scope.close();
-    }
-  }
-
-  public long addCacheDirective(
-      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("addCacheDirective", traceSampler);
-    try {
-      return namenode.addCacheDirective(info, flags);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException();
-    } finally {
-      scope.close();
-    }
-  }
-  
-  public void modifyCacheDirective(
-      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
-    checkOpen();
-    TraceScope scope = Trace.startSpan("modifyCacheDirect

<TRUNCATED>

[15/58] [abbrv] hadoop git commit: HDFS-8740. Move DistributedFileSystem to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c030c6e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
deleted file mode 100644
index 1d20f82..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ /dev/null
@@ -1,2262 +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.net.InetSocketAddress;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-
-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.BlockStoragePolicySpi;
-import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSLinkResolver;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileChecksum;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileSystemLinkResolver;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.FsStatus;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.Options.ChecksumOpt;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
-import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
-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.CachePoolEntry;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-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.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.HdfsLocatedFileStatus;
-import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
-import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-
-/****************************************************************
- * Implementation of the abstract FileSystem for the DFS system.
- * This object is the way end-user code interacts with a Hadoop
- * DistributedFileSystem.
- *
- *****************************************************************/
-@InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
-@InterfaceStability.Unstable
-public class DistributedFileSystem extends FileSystem {
-  private Path workingDir;
-  private URI uri;
-  private String homeDirPrefix =
-      HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT;
-
-  DFSClient dfs;
-  private boolean verifyChecksum = true;
-  
-  static{
-    HdfsConfiguration.init();
-  }
-
-  public DistributedFileSystem() {
-  }
-
-  /**
-   * Return the protocol scheme for the FileSystem.
-   * <p/>
-   *
-   * @return <code>hdfs</code>
-   */
-  @Override
-  public String getScheme() {
-    return HdfsConstants.HDFS_URI_SCHEME;
-  }
-
-  @Override
-  public URI getUri() { return uri; }
-
-  @Override
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    super.initialize(uri, conf);
-    setConf(conf);
-
-    String host = uri.getHost();
-    if (host == null) {
-      throw new IOException("Incomplete HDFS URI, no host: "+ uri);
-    }
-    homeDirPrefix = conf.get(
-        HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_KEY,
-        HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT);
-
-    this.dfs = new DFSClient(uri, conf, statistics);
-    this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
-    this.workingDir = getHomeDirectory();
-  }
-
-  @Override
-  public Path getWorkingDirectory() {
-    return workingDir;
-  }
-
-  @Override
-  public long getDefaultBlockSize() {
-    return dfs.getConf().getDefaultBlockSize();
-  }
-
-  @Override
-  public short getDefaultReplication() {
-    return dfs.getConf().getDefaultReplication();
-  }
-
-  @Override
-  public void setWorkingDirectory(Path dir) {
-    String result = fixRelativePart(dir).toUri().getPath();
-    if (!DFSUtil.isValidName(result)) {
-      throw new IllegalArgumentException("Invalid DFS directory name " + 
-                                         result);
-    }
-    workingDir = fixRelativePart(dir);
-  }
-
-  @Override
-  public Path getHomeDirectory() {
-    return makeQualified(new Path(homeDirPrefix + "/"
-        + dfs.ugi.getShortUserName()));
-  }
-
-  /**
-   * Checks that the passed URI belongs to this filesystem and returns
-   * just the path component. Expects a URI with an absolute path.
-   * 
-   * @param file URI with absolute path
-   * @return path component of {file}
-   * @throws IllegalArgumentException if URI does not belong to this DFS
-   */
-  private String getPathName(Path file) {
-    checkPath(file);
-    String result = file.toUri().getPath();
-    if (!DFSUtil.isValidName(result)) {
-      throw new IllegalArgumentException("Pathname " + result + " from " +
-                                         file+" is not a valid DFS filename.");
-    }
-    return result;
-  }
-  
-  @Override
-  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
-      long len) throws IOException {
-    if (file == null) {
-      return null;
-    }
-    return getFileBlockLocations(file.getPath(), start, len);
-  }
-  
-  @Override
-  public BlockLocation[] getFileBlockLocations(Path p, 
-      final long start, final long len) throws IOException {
-    statistics.incrementReadOps(1);
-    final Path absF = fixRelativePart(p);
-    return new FileSystemLinkResolver<BlockLocation[]>() {
-      @Override
-      public BlockLocation[] doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.getBlockLocations(getPathName(p), start, len);
-      }
-      @Override
-      public BlockLocation[] next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.getFileBlockLocations(p, start, len);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public void setVerifyChecksum(boolean verifyChecksum) {
-    this.verifyChecksum = verifyChecksum;
-  }
-
-  /** 
-   * Start the lease recovery of a file
-   *
-   * @param f a file
-   * @return true if the file is already closed
-   * @throws IOException if an error occurs
-   */
-  public boolean recoverLease(final Path f) throws IOException {
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<Boolean>() {
-      @Override
-      public Boolean doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.recoverLease(getPathName(p));
-      }
-      @Override
-      public Boolean next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          return myDfs.recoverLease(p);
-        }
-        throw new UnsupportedOperationException("Cannot recoverLease through" +
-            " a symlink to a non-DistributedFileSystem: " + f + " -> " + p);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public FSDataInputStream open(Path f, final int bufferSize)
-      throws IOException {
-    statistics.incrementReadOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FSDataInputStream>() {
-      @Override
-      public FSDataInputStream doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        final DFSInputStream dfsis =
-          dfs.open(getPathName(p), bufferSize, verifyChecksum);
-        return dfs.createWrappedInputStream(dfsis);
-      }
-      @Override
-      public FSDataInputStream next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.open(p, bufferSize);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public FSDataOutputStream append(Path f, final int bufferSize,
-      final Progressable progress) throws IOException {
-    return append(f, EnumSet.of(CreateFlag.APPEND), bufferSize, progress);
-  }
-
-  /**
-   * Append to an existing file (optional operation).
-   * 
-   * @param f the existing file to be appended.
-   * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
-   *          to be present.
-   * @param bufferSize the size of the buffer to be used.
-   * @param progress for reporting progress if it is not null.
-   * @return Returns instance of {@link FSDataOutputStream}
-   * @throws IOException
-   */
-  public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
-      final int bufferSize, final Progressable progress) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FSDataOutputStream>() {
-      @Override
-      public FSDataOutputStream doCall(final Path p)
-          throws IOException {
-        return dfs.append(getPathName(p), bufferSize, flag, progress,
-            statistics);
-      }
-      @Override
-      public FSDataOutputStream next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.append(p, bufferSize);
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * Append to an existing file (optional operation).
-   * 
-   * @param f the existing file to be appended.
-   * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
-   *          to be present.
-   * @param bufferSize the size of the buffer to be used.
-   * @param progress for reporting progress if it is not null.
-   * @param favoredNodes Favored nodes for new blocks
-   * @return Returns instance of {@link FSDataOutputStream}
-   * @throws IOException
-   */
-  public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
-      final int bufferSize, final Progressable progress,
-      final InetSocketAddress[] favoredNodes) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FSDataOutputStream>() {
-      @Override
-      public FSDataOutputStream doCall(final Path p)
-          throws IOException {
-        return dfs.append(getPathName(p), bufferSize, flag, progress,
-            statistics, favoredNodes);
-      }
-      @Override
-      public FSDataOutputStream next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.append(p, bufferSize);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public FSDataOutputStream create(Path f, FsPermission permission,
-      boolean overwrite, int bufferSize, short replication, long blockSize,
-      Progressable progress) throws IOException {
-    return this.create(f, permission,
-        overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
-            : EnumSet.of(CreateFlag.CREATE), bufferSize, replication,
-        blockSize, progress, null);
-  }
-
-  /**
-   * Same as  
-   * {@link #create(Path, FsPermission, boolean, int, short, long, 
-   * Progressable)} 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. And with favored nodes, blocks will be pinned
-   * on the datanodes to prevent balancing move the block. HDFS could move the
-   * blocks during replication, to move the blocks from favored nodes. A value
-   * of null means no favored nodes for this create
-   */
-  public HdfsDataOutputStream create(final Path f,
-      final FsPermission permission, final boolean overwrite,
-      final int bufferSize, final short replication, final long blockSize,
-      final Progressable progress, final InetSocketAddress[] favoredNodes)
-          throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<HdfsDataOutputStream>() {
-      @Override
-      public HdfsDataOutputStream doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        final DFSOutputStream out = dfs.create(getPathName(f), permission,
-            overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
-                : EnumSet.of(CreateFlag.CREATE),
-            true, replication, blockSize, progress, bufferSize, null,
-            favoredNodes);
-        return dfs.createWrappedOutputStream(out, statistics);
-      }
-      @Override
-      public HdfsDataOutputStream next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          return myDfs.create(p, permission, overwrite, bufferSize, replication,
-              blockSize, progress, favoredNodes);
-        }
-        throw new UnsupportedOperationException("Cannot create with" +
-            " favoredNodes through a symlink to a non-DistributedFileSystem: "
-            + f + " -> " + p);
-      }
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public FSDataOutputStream create(final Path f, final FsPermission permission,
-    final EnumSet<CreateFlag> cflags, final int bufferSize,
-    final short replication, final long blockSize, final Progressable progress,
-    final ChecksumOpt checksumOpt) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FSDataOutputStream>() {
-      @Override
-      public FSDataOutputStream doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
-                cflags, replication, blockSize, progress, bufferSize,
-                checksumOpt);
-        return dfs.createWrappedOutputStream(dfsos, statistics);
-      }
-      @Override
-      public FSDataOutputStream next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.create(p, permission, cflags, bufferSize,
-            replication, blockSize, progress, checksumOpt);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  protected HdfsDataOutputStream primitiveCreate(Path f,
-    FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
-    short replication, long blockSize, Progressable progress,
-    ChecksumOpt checksumOpt) throws IOException {
-    statistics.incrementWriteOps(1);
-    final DFSOutputStream dfsos = dfs.primitiveCreate(
-      getPathName(fixRelativePart(f)),
-      absolutePermission, flag, true, replication, blockSize,
-      progress, bufferSize, checksumOpt);
-    return dfs.createWrappedOutputStream(dfsos, statistics);
-  }
-
-  /**
-   * Same as create(), except fails if parent directory doesn't already exist.
-   */
-  @Override
-  @SuppressWarnings("deprecation")
-  public FSDataOutputStream createNonRecursive(final Path f,
-      final FsPermission permission, final EnumSet<CreateFlag> flag,
-      final int bufferSize, final short replication, final long blockSize,
-      final Progressable progress) throws IOException {
-    statistics.incrementWriteOps(1);
-    if (flag.contains(CreateFlag.OVERWRITE)) {
-      flag.add(CreateFlag.CREATE);
-    }
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FSDataOutputStream>() {
-      @Override
-      public FSDataOutputStream doCall(final Path p) throws IOException,
-          UnresolvedLinkException {
-        final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
-          flag, false, replication, blockSize, progress, bufferSize, null);
-        return dfs.createWrappedOutputStream(dfsos, statistics);
-      }
-
-      @Override
-      public FSDataOutputStream next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.createNonRecursive(p, permission, flag, bufferSize,
-            replication, blockSize, progress);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public boolean setReplication(Path src, 
-                                final short replication
-                               ) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(src);
-    return new FileSystemLinkResolver<Boolean>() {
-      @Override
-      public Boolean doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.setReplication(getPathName(p), replication);
-      }
-      @Override
-      public Boolean next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.setReplication(p, replication);
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * Set the source path to the specified storage policy.
-   *
-   * @param src The source path referring to either a directory or a file.
-   * @param policyName The name of the storage policy.
-   */
-  @Override
-  public void setStoragePolicy(final Path src, final String policyName)
-      throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(src);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.setStoragePolicy(getPathName(p), policyName);
-        return null;
-      }
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        fs.setStoragePolicy(p, policyName);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException {
-    statistics.incrementReadOps(1);
-    Path absF = fixRelativePart(path);
-
-    return new FileSystemLinkResolver<BlockStoragePolicySpi>() {
-      @Override
-      public BlockStoragePolicySpi doCall(final Path p) throws IOException {
-        return getClient().getStoragePolicy(getPathName(p));
-      }
-
-      @Override
-      public BlockStoragePolicySpi next(final FileSystem fs, final Path p)
-          throws IOException, UnresolvedLinkException {
-        return fs.getStoragePolicy(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public Collection<BlockStoragePolicy> getAllStoragePolicies()
-      throws IOException {
-    return Arrays.asList(dfs.getStoragePolicies());
-  }
-
-  /**
-   * Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
-   * @return
-   * @throws IOException
-   */
-  @Deprecated
-  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    statistics.incrementReadOps(1);
-    return dfs.getStoragePolicies();
-  }
-
-  /**
-   * Move blocks from srcs to trg and delete srcs afterwards.
-   * The file block sizes must be the same.
-   * 
-   * @param trg existing file to append to
-   * @param psrcs list of files (same block size, same replication)
-   * @throws IOException
-   */
-  @Override
-  public void concat(Path trg, Path [] psrcs) throws IOException {
-    statistics.incrementWriteOps(1);
-    // Make target absolute
-    Path absF = fixRelativePart(trg);
-    // Make all srcs absolute
-    Path[] srcs = new Path[psrcs.length];
-    for (int i=0; i<psrcs.length; i++) {
-      srcs[i] = fixRelativePart(psrcs[i]);
-    }
-    // Try the concat without resolving any links
-    String[] srcsStr = new String[psrcs.length];
-    try {
-      for (int i=0; i<psrcs.length; i++) {
-        srcsStr[i] = getPathName(srcs[i]);
-      }
-      dfs.concat(getPathName(trg), srcsStr);
-    } catch (UnresolvedLinkException e) {
-      // Exception could be from trg or any src.
-      // Fully resolve trg and srcs. Fail if any of them are a symlink.
-      FileStatus stat = getFileLinkStatus(absF);
-      if (stat.isSymlink()) {
-        throw new IOException("Cannot concat with a symlink target: "
-            + trg + " -> " + stat.getPath());
-      }
-      absF = fixRelativePart(stat.getPath());
-      for (int i=0; i<psrcs.length; i++) {
-        stat = getFileLinkStatus(srcs[i]);
-        if (stat.isSymlink()) {
-          throw new IOException("Cannot concat with a symlink src: "
-              + psrcs[i] + " -> " + stat.getPath());
-        }
-        srcs[i] = fixRelativePart(stat.getPath());
-      }
-      // Try concat again. Can still race with another symlink.
-      for (int i=0; i<psrcs.length; i++) {
-        srcsStr[i] = getPathName(srcs[i]);
-      }
-      dfs.concat(getPathName(absF), srcsStr);
-    }
-  }
-
-  
-  @SuppressWarnings("deprecation")
-  @Override
-  public boolean rename(Path src, Path dst) throws IOException {
-    statistics.incrementWriteOps(1);
-
-    final Path absSrc = fixRelativePart(src);
-    final Path absDst = fixRelativePart(dst);
-
-    // Try the rename without resolving first
-    try {
-      return dfs.rename(getPathName(absSrc), getPathName(absDst));
-    } catch (UnresolvedLinkException e) {
-      // Fully resolve the source
-      final Path source = getFileLinkStatus(absSrc).getPath();
-      // Keep trying to resolve the destination
-      return new FileSystemLinkResolver<Boolean>() {
-        @Override
-        public Boolean doCall(final Path p)
-            throws IOException, UnresolvedLinkException {
-          return dfs.rename(getPathName(source), getPathName(p));
-        }
-        @Override
-        public Boolean next(final FileSystem fs, final Path p)
-            throws IOException {
-          // Should just throw an error in FileSystem#checkPath
-          return doCall(p);
-        }
-      }.resolve(this, absDst);
-    }
-  }
-
-  /** 
-   * This rename operation is guaranteed to be atomic.
-   */
-  @SuppressWarnings("deprecation")
-  @Override
-  public void rename(Path src, Path dst, final Options.Rename... options)
-      throws IOException {
-    statistics.incrementWriteOps(1);
-    final Path absSrc = fixRelativePart(src);
-    final Path absDst = fixRelativePart(dst);
-    // Try the rename without resolving first
-    try {
-      dfs.rename(getPathName(absSrc), getPathName(absDst), options);
-    } catch (UnresolvedLinkException e) {
-      // Fully resolve the source
-      final Path source = getFileLinkStatus(absSrc).getPath();
-      // Keep trying to resolve the destination
-      new FileSystemLinkResolver<Void>() {
-        @Override
-        public Void doCall(final Path p)
-            throws IOException, UnresolvedLinkException {
-          dfs.rename(getPathName(source), getPathName(p), options);
-          return null;
-        }
-        @Override
-        public Void next(final FileSystem fs, final Path p)
-            throws IOException {
-          // Should just throw an error in FileSystem#checkPath
-          return doCall(p);
-        }
-      }.resolve(this, absDst);
-    }
-  }
-
-  @Override
-  public boolean truncate(Path f, final long newLength) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<Boolean>() {
-      @Override
-      public Boolean doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.truncate(getPathName(p), newLength);
-      }
-      @Override
-      public Boolean next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.truncate(p, newLength);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public boolean delete(Path f, final boolean recursive) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<Boolean>() {
-      @Override
-      public Boolean doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.delete(getPathName(p), recursive);
-      }
-      @Override
-      public Boolean next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.delete(p, recursive);
-      }
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public ContentSummary getContentSummary(Path f) throws IOException {
-    statistics.incrementReadOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<ContentSummary>() {
-      @Override
-      public ContentSummary doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.getContentSummary(getPathName(p));
-      }
-      @Override
-      public ContentSummary next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.getContentSummary(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  /** Set a directory's quotas
-   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long, StorageType)
-   */
-  public void setQuota(Path src, final long namespaceQuota,
-      final long storagespaceQuota) throws IOException {
-    Path absF = fixRelativePart(src);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota);
-        return null;
-      }
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        // setQuota is not defined in FileSystem, so we only can resolve
-        // within this DFS
-        return doCall(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * Set the per type storage quota of a directory.
-   *
-   * @param src target directory whose quota is to be modified.
-   * @param type storage type of the specific storage type quota to be modified.
-   * @param quota value of the specific storage type quota to be modified.
-   * Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type.
-   */
-  public void setQuotaByStorageType(
-    Path src, final StorageType type, final long quota)
-    throws IOException {
-    Path absF = fixRelativePart(src);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-        throws IOException, UnresolvedLinkException {
-        dfs.setQuotaByStorageType(getPathName(p), type, quota);
-        return null;
-      }
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-        throws IOException {
-        // setQuotaByStorageType is not defined in FileSystem, so we only can resolve
-        // within this DFS
-        return doCall(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  private FileStatus[] listStatusInternal(Path p) throws IOException {
-    String src = getPathName(p);
-
-    // fetch the first batch of entries in the directory
-    DirectoryListing thisListing = dfs.listPaths(
-        src, HdfsFileStatus.EMPTY_NAME);
-
-    if (thisListing == null) { // the directory does not exist
-      throw new FileNotFoundException("File " + p + " does not exist.");
-    }
-    
-    HdfsFileStatus[] partialListing = thisListing.getPartialListing();
-    if (!thisListing.hasMore()) { // got all entries of the directory
-      FileStatus[] stats = new FileStatus[partialListing.length];
-      for (int i = 0; i < partialListing.length; i++) {
-        stats[i] = partialListing[i].makeQualified(getUri(), p);
-      }
-      statistics.incrementReadOps(1);
-      return stats;
-    }
-
-    // The directory size is too big that it needs to fetch more
-    // estimate the total number of entries in the directory
-    int totalNumEntries =
-      partialListing.length + thisListing.getRemainingEntries();
-    ArrayList<FileStatus> listing =
-      new ArrayList<FileStatus>(totalNumEntries);
-    // add the first batch of entries to the array list
-    for (HdfsFileStatus fileStatus : partialListing) {
-      listing.add(fileStatus.makeQualified(getUri(), p));
-    }
-    statistics.incrementLargeReadOps(1);
- 
-    // now fetch more entries
-    do {
-      thisListing = dfs.listPaths(src, thisListing.getLastName());
- 
-      if (thisListing == null) { // the directory is deleted
-        throw new FileNotFoundException("File " + p + " does not exist.");
-      }
- 
-      partialListing = thisListing.getPartialListing();
-      for (HdfsFileStatus fileStatus : partialListing) {
-        listing.add(fileStatus.makeQualified(getUri(), p));
-      }
-      statistics.incrementLargeReadOps(1);
-    } while (thisListing.hasMore());
- 
-    return listing.toArray(new FileStatus[listing.size()]);
-  }
-
-  /**
-   * List all the entries of a directory
-   *
-   * Note that this operation is not atomic for a large directory.
-   * The entries of a directory may be fetched from NameNode multiple times.
-   * It only guarantees that  each name occurs once if a directory
-   * undergoes changes between the calls.
-   */
-  @Override
-  public FileStatus[] listStatus(Path p) throws IOException {
-    Path absF = fixRelativePart(p);
-    return new FileSystemLinkResolver<FileStatus[]>() {
-      @Override
-      public FileStatus[] doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return listStatusInternal(p);
-      }
-      @Override
-      public FileStatus[] next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.listStatus(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
-      final PathFilter filter)
-  throws IOException {
-    Path absF = fixRelativePart(p);
-    return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() {
-      @Override
-      public RemoteIterator<LocatedFileStatus> doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return new DirListingIterator<LocatedFileStatus>(p, filter, true);
-      }
-
-      @Override
-      public RemoteIterator<LocatedFileStatus> next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          return ((DistributedFileSystem)fs).listLocatedStatus(p, filter);
-        }
-        // symlink resolution for this methos does not work cross file systems
-        // because it is a protected method.
-        throw new IOException("Link resolution does not work with multiple " +
-            "file systems for listLocatedStatus(): " + p);
-      }
-    }.resolve(this, absF);
-  }
-
-
-  /**
-   * Returns a remote iterator so that followup calls are made on demand
-   * while consuming the entries. This reduces memory consumption during
-   * listing of a large directory.
-   *
-   * @param p target path
-   * @return remote iterator
-   */
-  @Override
-  public RemoteIterator<FileStatus> listStatusIterator(final Path p)
-  throws IOException {
-    Path absF = fixRelativePart(p);
-    return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() {
-      @Override
-      public RemoteIterator<FileStatus> doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return new DirListingIterator<FileStatus>(p, false);
-      }
-
-      @Override
-      public RemoteIterator<FileStatus> next(final FileSystem fs, final Path p)
-          throws IOException {
-          return ((DistributedFileSystem)fs).listStatusIterator(p);
-      }
-    }.resolve(this, absF);
-
-  }
-
-  /**
-   * This class defines an iterator that returns
-   * the file status of each file/subdirectory of a directory
-   * 
-   * if needLocation, status contains block location if it is a file
-   * throws a RuntimeException with the error as its cause.
-   * 
-   * @param <T> the type of the file status
-   */
-  private class  DirListingIterator<T extends FileStatus>
-  implements RemoteIterator<T> {
-    private DirectoryListing thisListing;
-    private int i;
-    private Path p;
-    private String src;
-    private T curStat = null;
-    private PathFilter filter;
-    private boolean needLocation;
-
-    private DirListingIterator(Path p, PathFilter filter,
-        boolean needLocation) throws IOException {
-      this.p = p;
-      this.src = getPathName(p);
-      this.filter = filter;
-      this.needLocation = needLocation;
-      // fetch the first batch of entries in the directory
-      thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME,
-          needLocation);
-      statistics.incrementReadOps(1);
-      if (thisListing == null) { // the directory does not exist
-        throw new FileNotFoundException("File " + p + " does not exist.");
-      }
-      i = 0;
-    }
-
-    private DirListingIterator(Path p, boolean needLocation)
-        throws IOException {
-      this(p, null, needLocation);
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public boolean hasNext() throws IOException {
-      while (curStat == null && hasNextNoFilter()) {
-        T next;
-        HdfsFileStatus fileStat = thisListing.getPartialListing()[i++];
-        if (needLocation) {
-          next = (T)((HdfsLocatedFileStatus)fileStat)
-              .makeQualifiedLocated(getUri(), p);
-        } else {
-          next = (T)fileStat.makeQualified(getUri(), p);
-        }
-          // apply filter if not null
-        if (filter == null || filter.accept(next.getPath())) {
-          curStat = next;
-        }
-      }
-      return curStat != null;
-    }
-      
-    /** Check if there is a next item before applying the given filter */
-    private boolean hasNextNoFilter() throws IOException {
-      if (thisListing == null) {
-        return false;
-      }
-      if (i >= thisListing.getPartialListing().length
-          && thisListing.hasMore()) { 
-        // current listing is exhausted & fetch a new listing
-        thisListing = dfs.listPaths(src, thisListing.getLastName(),
-            needLocation);
-        statistics.incrementReadOps(1);
-        if (thisListing == null) {
-          return false;
-        }
-        i = 0;
-      }
-      return (i < thisListing.getPartialListing().length);
-    }
-
-    @Override
-    public T next() throws IOException {
-      if (hasNext()) {
-        T tmp = curStat;
-        curStat = null;
-        return tmp;
-      } 
-      throw new java.util.NoSuchElementException("No more entry in " + p);
-    }
-  }
-  
-  /**
-   * Create a directory, only when the parent directories exist.
-   *
-   * See {@link FsPermission#applyUMask(FsPermission)} for details of how
-   * the permission is applied.
-   *
-   * @param f           The path to create
-   * @param permission  The permission.  See FsPermission#applyUMask for 
-   *                    details about how this is used to calculate the
-   *                    effective permission.
-   */
-  public boolean mkdir(Path f, FsPermission permission) throws IOException {
-    return mkdirsInternal(f, permission, false);
-  }
-
-  /**
-   * Create a directory and its parent directories.
-   *
-   * See {@link FsPermission#applyUMask(FsPermission)} for details of how
-   * the permission is applied.
-   *
-   * @param f           The path to create
-   * @param permission  The permission.  See FsPermission#applyUMask for 
-   *                    details about how this is used to calculate the
-   *                    effective permission.
-   */
-  @Override
-  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
-    return mkdirsInternal(f, permission, true);
-  }
-
-  private boolean mkdirsInternal(Path f, final FsPermission permission,
-      final boolean createParent) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<Boolean>() {
-      @Override
-      public Boolean doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.mkdirs(getPathName(p), permission, createParent);
-      }
-
-      @Override
-      public Boolean next(final FileSystem fs, final Path p)
-          throws IOException {
-        // FileSystem doesn't have a non-recursive mkdir() method
-        // Best we can do is error out
-        if (!createParent) {
-          throw new IOException("FileSystem does not support non-recursive"
-              + "mkdir");
-        }
-        return fs.mkdirs(p, permission);
-      }
-    }.resolve(this, absF);
-  }
-
-  @SuppressWarnings("deprecation")
-  @Override
-  protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
-    throws IOException {
-    statistics.incrementWriteOps(1);
-    return dfs.primitiveMkdir(getPathName(f), absolutePermission);
-  }
-
- 
-  @Override
-  public void close() throws IOException {
-    try {
-      dfs.closeOutputStreams(false);
-      super.close();
-    } finally {
-      dfs.close();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "DFS[" + dfs + "]";
-  }
-
-  @InterfaceAudience.Private
-  @VisibleForTesting
-  public DFSClient getClient() {
-    return dfs;
-  }        
-  
-  @Override
-  public FsStatus getStatus(Path p) throws IOException {
-    statistics.incrementReadOps(1);
-    return dfs.getDiskStatus();
-  }
-
-  /**
-   * Returns count of blocks with no good replicas left. Normally should be
-   * zero.
-   * 
-   * @throws IOException
-   */
-  public long getMissingBlocksCount() throws IOException {
-    return dfs.getMissingBlocksCount();
-  }
-
-  /**
-   * Returns count of blocks with replication factor 1 and have
-   * lost the only replica.
-   *
-   * @throws IOException
-   */
-  public long getMissingReplOneBlocksCount() throws IOException {
-    return dfs.getMissingReplOneBlocksCount();
-  }
-
-  /**
-   * Returns count of blocks with one of more replica missing.
-   * 
-   * @throws IOException
-   */
-  public long getUnderReplicatedBlocksCount() throws IOException {
-    return dfs.getUnderReplicatedBlocksCount();
-  }
-
-  /**
-   * Returns count of blocks with at least one replica marked corrupt.
-   * 
-   * @throws IOException
-   */
-  public long getCorruptBlocksCount() throws IOException {
-    return dfs.getCorruptBlocksCount();
-  }
-
-  @Override
-  public RemoteIterator<Path> listCorruptFileBlocks(Path path)
-    throws IOException {
-    return new CorruptFileBlockIterator(dfs, path);
-  }
-
-  /** @return datanode statistics. */
-  public DatanodeInfo[] getDataNodeStats() throws IOException {
-    return getDataNodeStats(DatanodeReportType.ALL);
-  }
-
-  /** @return datanode statistics for the given type. */
-  public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type
-      ) throws IOException {
-    return dfs.datanodeReport(type);
-  }
-
-  /**
-   * Enter, leave or get safe mode.
-   *  
-   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
-   *    HdfsConstants.SafeModeAction,boolean)
-   */
-  public boolean setSafeMode(HdfsConstants.SafeModeAction action) 
-  throws IOException {
-    return setSafeMode(action, false);
-  }
-
-  /**
-   * Enter, leave or get safe mode.
-   * 
-   * @param action
-   *          One of SafeModeAction.ENTER, SafeModeAction.LEAVE and
-   *          SafeModeAction.GET
-   * @param isChecked
-   *          If true check only for Active NNs status, else check first NN's
-   *          status
-   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(SafeModeAction, boolean)
-   */
-  public boolean setSafeMode(HdfsConstants.SafeModeAction action,
-      boolean isChecked) throws IOException {
-    return dfs.setSafeMode(action, isChecked);
-  }
-
-  /**
-   * Save namespace image.
-   *
-   * @param timeWindow NameNode can ignore this command if the latest
-   *                   checkpoint was done within the given time period (in
-   *                   seconds).
-   * @return true if a new checkpoint has been made
-   * @see ClientProtocol#saveNamespace(long, long)
-   */
-  public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
-    return dfs.saveNamespace(timeWindow, txGap);
-  }
-
-  /**
-   * Save namespace image. NameNode always does the checkpoint.
-   */
-  public void saveNamespace() throws IOException {
-    saveNamespace(0, 0);
-  }
-
-  /**
-   * Rolls the edit log on the active NameNode.
-   * Requires super-user privileges.
-   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits()
-   * @return the transaction ID of the newly created segment
-   */
-  public long rollEdits() throws AccessControlException, IOException {
-    return dfs.rollEdits();
-  }
-
-  /**
-   * enable/disable/check restoreFaileStorage
-   * 
-   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
-   */
-  public boolean restoreFailedStorage(String arg)
-      throws AccessControlException, IOException {
-    return dfs.restoreFailedStorage(arg);
-  }
-  
-
-  /**
-   * Refreshes the list of hosts and excluded hosts from the configured 
-   * files.  
-   */
-  public void refreshNodes() throws IOException {
-    dfs.refreshNodes();
-  }
-
-  /**
-   * Finalize previously upgraded files system state.
-   * @throws IOException
-   */
-  public void finalizeUpgrade() throws IOException {
-    dfs.finalizeUpgrade();
-  }
-
-  /**
-   * Rolling upgrade: prepare/finalize/query.
-   */
-  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
-      throws IOException {
-    return dfs.rollingUpgrade(action);
-  }
-
-  /*
-   * Requests the namenode to dump data strcutures into specified 
-   * file.
-   */
-  public void metaSave(String pathname) throws IOException {
-    dfs.metaSave(pathname);
-  }
-
-  @Override
-  public FsServerDefaults getServerDefaults() throws IOException {
-    return dfs.getServerDefaults();
-  }
-
-  /**
-   * Returns the stat information about the file.
-   * @throws FileNotFoundException if the file does not exist.
-   */
-  @Override
-  public FileStatus getFileStatus(Path f) throws IOException {
-    statistics.incrementReadOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FileStatus>() {
-      @Override
-      public FileStatus doCall(final Path p) throws IOException,
-          UnresolvedLinkException {
-        HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
-        if (fi != null) {
-          return fi.makeQualified(getUri(), p);
-        } else {
-          throw new FileNotFoundException("File does not exist: " + p);
-        }
-      }
-      @Override
-      public FileStatus next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.getFileStatus(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  @SuppressWarnings("deprecation")
-  @Override
-  public void createSymlink(final Path target, final Path link,
-      final boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, UnsupportedFileSystemException, 
-      IOException {
-    if (!FileSystem.areSymlinksEnabled()) {
-      throw new UnsupportedOperationException("Symlinks not supported");
-    }
-    statistics.incrementWriteOps(1);
-    final Path absF = fixRelativePart(link);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException,
-          UnresolvedLinkException {
-        dfs.createSymlink(target.toString(), getPathName(p), createParent);
-        return null;
-      }
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException, UnresolvedLinkException {
-        fs.createSymlink(target, p, createParent);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public boolean supportsSymlinks() {
-    return true;
-  }
-
-  @Override
-  public FileStatus getFileLinkStatus(final Path f)
-      throws AccessControlException, FileNotFoundException,
-      UnsupportedFileSystemException, IOException {
-    statistics.incrementReadOps(1);
-    final Path absF = fixRelativePart(f);
-    FileStatus status = new FileSystemLinkResolver<FileStatus>() {
-      @Override
-      public FileStatus doCall(final Path p) throws IOException,
-          UnresolvedLinkException {
-        HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
-        if (fi != null) {
-          return fi.makeQualified(getUri(), p);
-        } else {
-          throw new FileNotFoundException("File does not exist: " + p);
-        }
-      }
-      @Override
-      public FileStatus next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        return fs.getFileLinkStatus(p);
-      }
-    }.resolve(this, absF);
-    // Fully-qualify the symlink
-    if (status.isSymlink()) {
-      Path targetQual = FSLinkResolver.qualifySymlinkTarget(this.getUri(),
-          status.getPath(), status.getSymlink());
-      status.setSymlink(targetQual);
-    }
-    return status;
-  }
-
-  @Override
-  public Path getLinkTarget(final Path f) throws AccessControlException,
-      FileNotFoundException, UnsupportedFileSystemException, IOException {
-    statistics.incrementReadOps(1);
-    final Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<Path>() {
-      @Override
-      public Path doCall(final Path p) throws IOException,
-          UnresolvedLinkException {
-        HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
-        if (fi != null) {
-          return fi.makeQualified(getUri(), p).getSymlink();
-        } else {
-          throw new FileNotFoundException("File does not exist: " + p);
-        }
-      }
-      @Override
-      public Path next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        return fs.getLinkTarget(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  protected Path resolveLink(Path f) throws IOException {
-    statistics.incrementReadOps(1);
-    String target = dfs.getLinkTarget(getPathName(fixRelativePart(f)));
-    if (target == null) {
-      throw new FileNotFoundException("File does not exist: " + f.toString());
-    }
-    return new Path(target);
-  }
-
-  @Override
-  public FileChecksum getFileChecksum(Path f) throws IOException {
-    statistics.incrementReadOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FileChecksum>() {
-      @Override
-      public FileChecksum doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
-      }
-
-      @Override
-      public FileChecksum next(final FileSystem fs, final Path p)
-          throws IOException {
-        return fs.getFileChecksum(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public FileChecksum getFileChecksum(Path f, final long length)
-      throws IOException {
-    statistics.incrementReadOps(1);
-    Path absF = fixRelativePart(f);
-    return new FileSystemLinkResolver<FileChecksum>() {
-      @Override
-      public FileChecksum doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.getFileChecksum(getPathName(p), length);
-      }
-
-      @Override
-      public FileChecksum next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          return ((DistributedFileSystem) fs).getFileChecksum(p, length);
-        } else {
-          throw new UnsupportedFileSystemException(
-              "getFileChecksum(Path, long) is not supported by "
-                  + fs.getClass().getSimpleName()); 
-        }
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public void setPermission(Path p, final FsPermission permission
-      ) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(p);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.setPermission(getPathName(p), permission);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        fs.setPermission(p, permission);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public void setOwner(Path p, final String username, final String groupname
-      ) throws IOException {
-    if (username == null && groupname == null) {
-      throw new IOException("username == null && groupname == null");
-    }
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(p);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.setOwner(getPathName(p), username, groupname);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        fs.setOwner(p, username, groupname);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public void setTimes(Path p, final long mtime, final long atime
-      ) throws IOException {
-    statistics.incrementWriteOps(1);
-    Path absF = fixRelativePart(p);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.setTimes(getPathName(p), mtime, atime);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        fs.setTimes(p, mtime, atime);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-  
-
-  @Override
-  protected int getDefaultPort() {
-    return HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
-      throws IOException {
-    Token<DelegationTokenIdentifier> result =
-      dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
-    return result;
-  }
-
-  /**
-   * Requests the namenode to tell all datanodes to use a new, non-persistent
-   * bandwidth value for dfs.balance.bandwidthPerSec.
-   * The bandwidth parameter is the max number of bytes per second of network
-   * bandwidth to be used by a datanode during balancing.
-   *
-   * @param bandwidth Balancer bandwidth in bytes per second for all datanodes.
-   * @throws IOException
-   */
-  public void setBalancerBandwidth(long bandwidth) throws IOException {
-    dfs.setBalancerBandwidth(bandwidth);
-  }
-
-  /**
-   * Get a canonical service name for this file system. If the URI is logical,
-   * the hostname part of the URI will be returned.
-   * @return a service string that uniquely identifies this file system.
-   */
-  @Override
-  public String getCanonicalServiceName() {
-    return dfs.getCanonicalServiceName();
-  }
-  
-  @Override
-  protected URI canonicalizeUri(URI uri) {
-    if (HAUtilClient.isLogicalUri(getConf(), uri)) {
-      // Don't try to DNS-resolve logical URIs, since the 'authority'
-      // portion isn't a proper hostname
-      return uri;
-    } else {
-      return NetUtils.getCanonicalUri(uri, getDefaultPort());
-    }
-  }
-
-  /**
-   * Utility function that returns if the NameNode is in safemode or not. In HA
-   * mode, this API will return only ActiveNN's safemode status.
-   * 
-   * @return true if NameNode is in safemode, false otherwise.
-   * @throws IOException
-   *           when there is an issue communicating with the NameNode
-   */
-  public boolean isInSafeMode() throws IOException {
-    return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
-  }
-
-  /** @see HdfsAdmin#allowSnapshot(Path) */
-  public void allowSnapshot(final Path path) throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.allowSnapshot(getPathName(p));
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          myDfs.allowSnapshot(p);
-        } else {
-          throw new UnsupportedOperationException("Cannot perform snapshot"
-              + " operations on a symlink to a non-DistributedFileSystem: "
-              + path + " -> " + p);
-        }
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-  
-  /** @see HdfsAdmin#disallowSnapshot(Path) */
-  public void disallowSnapshot(final Path path) throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.disallowSnapshot(getPathName(p));
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          myDfs.disallowSnapshot(p);
-        } else {
-          throw new UnsupportedOperationException("Cannot perform snapshot"
-              + " operations on a symlink to a non-DistributedFileSystem: "
-              + path + " -> " + p);
-        }
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public Path createSnapshot(final Path path, final String snapshotName) 
-      throws IOException {
-    Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<Path>() {
-      @Override
-      public Path doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return new Path(dfs.createSnapshot(getPathName(p), snapshotName));
-      }
-
-      @Override
-      public Path next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          return myDfs.createSnapshot(p);
-        } else {
-          throw new UnsupportedOperationException("Cannot perform snapshot"
-              + " operations on a symlink to a non-DistributedFileSystem: "
-              + path + " -> " + p);
-        }
-      }
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public void renameSnapshot(final Path path, final String snapshotOldName,
-      final String snapshotNewName) throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          myDfs.renameSnapshot(p, snapshotOldName, snapshotNewName);
-        } else {
-          throw new UnsupportedOperationException("Cannot perform snapshot"
-              + " operations on a symlink to a non-DistributedFileSystem: "
-              + path + " -> " + p);
-        }
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-  
-  /**
-   * @return All the snapshottable directories
-   * @throws IOException
-   */
-  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
-      throws IOException {
-    return dfs.getSnapshottableDirListing();
-  }
-  
-  @Override
-  public void deleteSnapshot(final Path snapshotDir, final String snapshotName)
-      throws IOException {
-    Path absF = fixRelativePart(snapshotDir);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        dfs.deleteSnapshot(getPathName(p), snapshotName);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          myDfs.deleteSnapshot(p, snapshotName);
-        } else {
-          throw new UnsupportedOperationException("Cannot perform snapshot"
-              + " operations on a symlink to a non-DistributedFileSystem: "
-              + snapshotDir + " -> " + p);
-        }
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * Get the difference between two snapshots, or between a snapshot and the
-   * current tree of a directory.
-   * 
-   * @see DFSClient#getSnapshotDiffReport(String, String, String)
-   */
-  public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
-      final String fromSnapshot, final String toSnapshot) throws IOException {
-    Path absF = fixRelativePart(snapshotDir);
-    return new FileSystemLinkResolver<SnapshotDiffReport>() {
-      @Override
-      public SnapshotDiffReport doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
-            toSnapshot);
-      }
-
-      @Override
-      public SnapshotDiffReport next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          myDfs.getSnapshotDiffReport(p, fromSnapshot, toSnapshot);
-        } else {
-          throw new UnsupportedOperationException("Cannot perform snapshot"
-              + " operations on a symlink to a non-DistributedFileSystem: "
-              + snapshotDir + " -> " + p);
-        }
-        return null;
-      }
-    }.resolve(this, absF);
-  }
- 
-  /**
-   * Get the close status of a file
-   * @param src The path to the file
-   *
-   * @return return true if file is closed
-   * @throws FileNotFoundException if the file does not exist.
-   * @throws IOException If an I/O error occurred     
-   */
-  public boolean isFileClosed(final Path src) throws IOException {
-    Path absF = fixRelativePart(src);
-    return new FileSystemLinkResolver<Boolean>() {
-      @Override
-      public Boolean doCall(final Path p)
-          throws IOException, UnresolvedLinkException {
-        return dfs.isFileClosed(getPathName(p));
-      }
-
-      @Override
-      public Boolean next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
-          return myDfs.isFileClosed(p);
-        } else {
-          throw new UnsupportedOperationException("Cannot call isFileClosed"
-              + " on a symlink to a non-DistributedFileSystem: "
-              + src + " -> " + p);
-        }
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * @see {@link #addCacheDirective(CacheDirectiveInfo, EnumSet)}
-   */
-  public long addCacheDirective(CacheDirectiveInfo info) throws IOException {
-    return addCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
-  }
-
-  /**
-   * Add a new CacheDirective.
-   * 
-   * @param info Information about a directive to add.
-   * @param flags {@link CacheFlag}s to use for this operation.
-   * @return the ID of the directive that was created.
-   * @throws IOException if the directive could not be added
-   */
-  public long addCacheDirective(
-      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
-    Preconditions.checkNotNull(info.getPath());
-    Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
-        makeQualified(getUri(), getWorkingDirectory());
-    return dfs.addCacheDirective(
-        new CacheDirectiveInfo.Builder(info).
-            setPath(path).
-            build(),
-        flags);
-  }
-
-  /**
-   * @see {@link #modifyCacheDirective(CacheDirectiveInfo, EnumSet)}
-   */
-  public void modifyCacheDirective(CacheDirectiveInfo info) throws IOException {
-    modifyCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
-  }
-
-  /**
-   * Modify a CacheDirective.
-   * 
-   * @param info Information about the directive to modify. You must set the ID
-   *          to indicate which CacheDirective you want to modify.
-   * @param flags {@link CacheFlag}s to use for this operation.
-   * @throws IOException if the directive could not be modified
-   */
-  public void modifyCacheDirective(
-      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
-    if (info.getPath() != null) {
-      info = new CacheDirectiveInfo.Builder(info).
-          setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
-              makeQualified(getUri(), getWorkingDirectory())).build();
-    }
-    dfs.modifyCacheDirective(info, flags);
-  }
-
-  /**
-   * Remove a CacheDirectiveInfo.
-   * 
-   * @param id identifier of the CacheDirectiveInfo to remove
-   * @throws IOException if the directive could not be removed
-   */
-  public void removeCacheDirective(long id)
-      throws IOException {
-    dfs.removeCacheDirective(id);
-  }
-  
-  /**
-   * List cache directives.  Incrementally fetches results from the server.
-   * 
-   * @param filter Filter parameters to use when listing the directives, null to
-   *               list all directives visible to us.
-   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
-   */
-  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
-      CacheDirectiveInfo filter) throws IOException {
-    if (filter == null) {
-      filter = new CacheDirectiveInfo.Builder().build();
-    }
-    if (filter.getPath() != null) {
-      filter = new CacheDirectiveInfo.Builder(filter).
-          setPath(new Path(getPathName(fixRelativePart(filter.getPath())))).
-          build();
-    }
-    final RemoteIterator<CacheDirectiveEntry> iter =
-        dfs.listCacheDirectives(filter);
-    return new RemoteIterator<CacheDirectiveEntry>() {
-      @Override
-      public boolean hasNext() throws IOException {
-        return iter.hasNext();
-      }
-
-      @Override
-      public CacheDirectiveEntry next() throws IOException {
-        // Although the paths we get back from the NameNode should always be
-        // absolute, we call makeQualified to add the scheme and authority of
-        // this DistributedFilesystem.
-        CacheDirectiveEntry desc = iter.next();
-        CacheDirectiveInfo info = desc.getInfo();
-        Path p = info.getPath().makeQualified(getUri(), getWorkingDirectory());
-        return new CacheDirectiveEntry(
-            new CacheDirectiveInfo.Builder(info).setPath(p).build(),
-            desc.getStats());
-      }
-    };
-  }
-
-  /**
-   * Add a cache pool.
-   *
-   * @param info
-   *          The request to add a cache pool.
-   * @throws IOException 
-   *          If the request could not be completed.
-   */
-  public void addCachePool(CachePoolInfo info) throws IOException {
-    CachePoolInfo.validate(info);
-    dfs.addCachePool(info);
-  }
-
-  /**
-   * Modify an existing cache pool.
-   *
-   * @param info
-   *          The request to modify a cache pool.
-   * @throws IOException 
-   *          If the request could not be completed.
-   */
-  public void modifyCachePool(CachePoolInfo info) throws IOException {
-    CachePoolInfo.validate(info);
-    dfs.modifyCachePool(info);
-  }
-    
-  /**
-   * Remove a cache pool.
-   *
-   * @param poolName
-   *          Name of the cache pool to remove.
-   * @throws IOException 
-   *          if the cache pool did not exist, or could not be removed.
-   */
-  public void removeCachePool(String poolName) throws IOException {
-    CachePoolInfo.validateName(poolName);
-    dfs.removeCachePool(poolName);
-  }
-
-  /**
-   * List all cache pools.
-   *
-   * @return A remote iterator from which you can get CachePoolEntry objects.
-   *          Requests will be made as needed.
-   * @throws IOException
-   *          If there was an error listing cache pools.
-   */
-  public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
-    return dfs.listCachePools();
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void modifyAclEntries(Path path, final List<AclEntry> aclSpec)
-      throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.modifyAclEntries(getPathName(p), aclSpec);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p) throws IOException {
-        fs.modifyAclEntries(p, aclSpec);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void removeAclEntries(Path path, final List<AclEntry> aclSpec)
-      throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.removeAclEntries(getPathName(p), aclSpec);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p) throws IOException {
-        fs.removeAclEntries(p, aclSpec);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void removeDefaultAcl(Path path) throws IOException {
-    final Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.removeDefaultAcl(getPathName(p));
-        return null;
-      }
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        fs.removeDefaultAcl(p);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void removeAcl(Path path) throws IOException {
-    final Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.removeAcl(getPathName(p));
-        return null;
-      }
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        fs.removeAcl(p);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void setAcl(Path path, final List<AclEntry> aclSpec) throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.setAcl(getPathName(p), aclSpec);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p) throws IOException {
-        fs.setAcl(p, aclSpec);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public AclStatus getAclStatus(Path path) throws IOException {
-    final Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<AclStatus>() {
-      @Override
-      public AclStatus doCall(final Path p) throws IOException {
-        return dfs.getAclStatus(getPathName(p));
-      }
-      @Override
-      public AclStatus next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        return fs.getAclStatus(p);
-      }
-    }.resolve(this, absF);
-  }
-  
-  /* HDFS only */
-  public void createEncryptionZone(final Path path, final String keyName)
-    throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException,
-          UnresolvedLinkException {
-        dfs.createEncryptionZone(getPathName(p), keyName);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p) throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
-          myDfs.createEncryptionZone(p, keyName);
-          return null;
-        } else {
-          throw new UnsupportedOperationException(
-              "Cannot call createEncryptionZone"
-                  + " on a symlink to a non-DistributedFileSystem: " + path
-                  + " -> " + p);
-        }
-      }
-    }.resolve(this, absF);
-  }
-
-  /* HDFS only */
-  public EncryptionZone getEZForPath(final Path path)
-          throws IOException {
-    Preconditions.checkNotNull(path);
-    Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<EncryptionZone>() {
-      @Override
-      public EncryptionZone doCall(final Path p) throws IOException,
-          UnresolvedLinkException {
-        return dfs.getEZForPath(getPathName(p));
-      }
-
-      @Override
-      public EncryptionZone next(final FileSystem fs, final Path p)
-          throws IOException {
-        if (fs instanceof DistributedFileSystem) {
-          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
-          return myDfs.getEZForPath(p);
-        } else {
-          throw new UnsupportedOperationException(
-              "Cannot call getEZForPath"
-                  + " on a symlink to a non-DistributedFileSystem: " + path
-                  + " -> " + p);
-        }
-      }
-    }.resolve(this, absF);
-  }
-
-  /* HDFS only */
-  public RemoteIterator<EncryptionZone> listEncryptionZones()
-      throws IOException {
-    return dfs.listEncryptionZones();
-  }
-
-  @Override
-  public void setXAttr(Path path, final String name, final byte[] value, 
-      final EnumSet<XAttrSetFlag> flag) throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.setXAttr(getPathName(p), name, value, flag);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p) throws IOException {
-        fs.setXAttr(p, name, value, flag);
-        return null;
-      }      
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public byte[] getXAttr(Path path, final String name) throws IOException {
-    final Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<byte[]>() {
-      @Override
-      public byte[] doCall(final Path p) throws IOException {
-        return dfs.getXAttr(getPathName(p), name);
-      }
-      @Override
-      public byte[] next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        return fs.getXAttr(p, name);
-      }
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public Map<String, byte[]> getXAttrs(Path path) throws IOException {
-    final Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<Map<String, byte[]>>() {
-      @Override
-      public Map<String, byte[]> doCall(final Path p) throws IOException {
-        return dfs.getXAttrs(getPathName(p));
-      }
-      @Override
-      public Map<String, byte[]> next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        return fs.getXAttrs(p);
-      }
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public Map<String, byte[]> getXAttrs(Path path, final List<String> names) 
-      throws IOException {
-    final Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<Map<String, byte[]>>() {
-      @Override
-      public Map<String, byte[]> doCall(final Path p) throws IOException {
-        return dfs.getXAttrs(getPathName(p), names);
-      }
-      @Override
-      public Map<String, byte[]> next(final FileSystem fs, final Path p)
-        throws IOException, UnresolvedLinkException {
-        return fs.getXAttrs(p, names);
-      }
-    }.resolve(this, absF);
-  }
-  
-  @Override
-  public List<String> listXAttrs(Path path)
-          throws IOException {
-    final Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<List<String>>() {
-      @Override
-      public List<String> doCall(final Path p) throws IOException {
-        return dfs.listXAttrs(getPathName(p));
-      }
-      @Override
-      public List<String> next(final FileSystem fs, final Path p)
-              throws IOException, UnresolvedLinkException {
-        return fs.listXAttrs(p);
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public void removeXAttr(Path path, final String name) throws IOException {
-    Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.removeXAttr(getPathName(p), name);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p) throws IOException {
-        fs.removeXAttr(p, name);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public void access(Path path, final FsAction mode) throws IOException {
-    final Path absF = fixRelativePart(path);
-    new FileSystemLinkResolver<Void>() {
-      @Override
-      public Void doCall(final Path p) throws IOException {
-        dfs.checkAccess(getPathName(p), mode);
-        return null;
-      }
-
-      @Override
-      public Void next(final FileSystem fs, final Path p)
-          throws IOException {
-        fs.access(p, mode);
-        return null;
-      }
-    }.resolve(this, absF);
-  }
-
-  @Override
-  public Token<?>[] addDelegationTokens(
-      final String renewer, Credentials credentials) throws IOException {
-    Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
-    if (dfs.isHDFSEncryptionEnabled()) {
-      KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension =
-          KeyProviderDelegationTokenExtension.
-              createKeyProviderDelegationTokenExtension(dfs.getKeyProvider());
-      Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
-          addDelegationTokens(renewer, credentials);
-      if (tokens != null && kpTokens != null) {
-        Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
-        System.arraycopy(tokens, 0, all, 0, tokens.length);
-        System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
-        tokens = all;
-      } else {
-        tokens = (tokens != null) ? tokens : kpTokens;
-      }
-    }
-    return tokens;
-  }
-
-  public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
-    return dfs.getInotifyEventStream();
-  }
-
-  public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
-      throws IOException {
-    return dfs.getInotifyEventStream(lastReadTxid);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c030c6e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
deleted file mode 100644
index 77bed1a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
+++ /dev/null
@@ -1,105 +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.util.NoSuchElementException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-
-/**
- * Provides an iterator interface for listCorruptFileBlocks.
- * This class is used by DistributedFileSystem and Hdfs.
- */
-@InterfaceAudience.Private
-public class CorruptFileBlockIterator implements RemoteIterator<Path> {
-  private final DFSClient dfs;
-  private final String path;
-
-  private String[] files = null;
-  private int fileIdx = 0;
-  private String cookie = null;
-  private Path nextPath = null;
-
-  private int callsMade = 0;
-
-  public CorruptFileBlockIterator(DFSClient dfs, Path path) throws IOException {
-    this.dfs = dfs;
-    this.path = path2String(path);
-    loadNext();
-  }
-
-  /**
-   * @return the number of calls made to the DFSClient.
-   * This is for debugging and testing purposes.
-   */
-  public int getCallsMade() {
-    return callsMade;
-  }
-
-  private String path2String(Path path) {
-    return path.toUri().getPath();
-  }
-
-  private Path string2Path(String string) {
-    return new Path(string);
-  }
-
-  private void loadNext() throws IOException {
-    if (files == null || fileIdx >= files.length) {
-      CorruptFileBlocks cfb = dfs.listCorruptFileBlocks(path, cookie);
-      files = cfb.getFiles();
-      cookie = cfb.getCookie();
-      fileIdx = 0;
-      callsMade++;
-    }
-
-    if (fileIdx >= files.length) {
-      // received an empty response
-      // there are no more corrupt file blocks
-      nextPath = null;
-    } else {
-      nextPath = string2Path(files[fileIdx]);
-      fileIdx++;
-    }
-  }
-
-  
-  @Override
-  public boolean hasNext() {
-    return nextPath != null;
-  }
-
-  
-  @Override
-  public Path next() throws IOException {
-    if (!hasNext()) {
-      throw new NoSuchElementException("No more corrupt file blocks");
-    }
-
-    Path result = nextPath;
-    loadNext();
-
-    return result;
-  }
-}
\ No newline at end of file


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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..de1d1ee
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -0,0 +1,918 @@
+/**
+ * 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;
+    }
+  }
+  
+  /**
+   * 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/bf37d3d8/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/bf37d3d8/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);


[30/58] [abbrv] hadoop git commit: HDFS-9092. Nfs silently drops overlapping write requests and causes data copying to fail. Contributed by Yongjun Zhang.

Posted by zh...@apache.org.
HDFS-9092. Nfs silently drops overlapping write requests and causes data copying to fail. Contributed by Yongjun Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: 151fca5032719e561226ef278e002739073c23ec
Parents: 5c3b663
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Mon Sep 28 18:45:00 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Sep 28 18:45:00 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/nfs/nfs3/OffsetRange.java       |   4 +
 .../hadoop/hdfs/nfs/nfs3/OpenFileCtx.java       | 141 +++++++++++--------
 .../apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java   |  82 ++++++++++-
 .../apache/hadoop/hdfs/nfs/nfs3/TestWrites.java |  92 +++++++++++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 5 files changed, 260 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/151fca50/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OffsetRange.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OffsetRange.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OffsetRange.java
index f02dcc0..764524a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OffsetRange.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OffsetRange.java
@@ -70,4 +70,8 @@ public class OffsetRange {
     }
     return false;
   }
+
+  public String toString() {
+    return "[" + getMin() + ", " + getMax() + ")";
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/151fca50/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
index 9610f48..9371a72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
@@ -490,11 +490,11 @@ class OpenFileCtx {
     int count = request.getCount();
     long smallerCount = offset + count - cachedOffset;
     if (LOG.isDebugEnabled()) {
-      LOG.debug(String.format("Got overwrite with appended data (%d-%d),"
-          + " current offset %d," + " drop the overlapped section (%d-%d)"
-          + " and append new data (%d-%d).", offset, (offset + count - 1),
-          cachedOffset, offset, (cachedOffset - 1), cachedOffset, (offset
-              + count - 1)));
+      LOG.debug(String.format("Got overwrite with appended data [%d-%d),"
+          + " current offset %d," + " drop the overlapped section [%d-%d)"
+          + " and append new data [%d-%d).", offset, (offset + count),
+          cachedOffset, offset, cachedOffset, cachedOffset, (offset
+              + count)));
     }
     
     ByteBuffer data = request.getData();
@@ -508,6 +508,22 @@ class OpenFileCtx {
     request.setCount((int) smallerCount);
   }
   
+  @VisibleForTesting
+  private static void trimWriteRequest(WriteCtx writeCtx,
+      long currentOffset) {
+    long offset = writeCtx.getOffset();
+    if (LOG.isDebugEnabled()) {
+      int count = writeCtx.getCount();
+      LOG.debug(String.format("Trim request [%d-%d),"
+          + " current offset %d," + " drop the overlapped section [%d-%d)"
+          + " and write new data [%d-%d)",
+          offset, (offset + count),
+          currentOffset, offset, (currentOffset),
+          currentOffset, (offset + count)));
+    }
+    writeCtx.trimWrite((int)(currentOffset - offset));
+  }
+
   /**
    * Creates and adds a WriteCtx into the pendingWrites map. This is a
    * synchronized method to handle concurrent writes.
@@ -527,23 +543,27 @@ class OpenFileCtx {
           + cachedOffset);
     }
 
-    // Handle a special case first
+    // Ignore write request with range below the current offset
+    if (offset + count <= cachedOffset) {
+      LOG.warn(String.format("Got overwrite [%d-%d) smaller than"
+          + " current offset %d," + " drop the request.",
+          offset, (offset + count), cachedOffset));
+      return null;
+    }
+
+    // Handle a special case: trim request whose offset is smaller than
+    // the current offset
     if ((offset < cachedOffset) && (offset + count > cachedOffset)) {
       // One Linux client behavior: after a file is closed and reopened to
       // write, the client sometimes combines previous written data(could still
       // be in kernel buffer) with newly appended data in one write. This is
       // usually the first write after file reopened. In this
       // case, we log the event and drop the overlapped section.
-      LOG.warn(String.format("Got overwrite with appended data (%d-%d),"
-          + " current offset %d," + " drop the overlapped section (%d-%d)"
-          + " and append new data (%d-%d).", offset, (offset + count - 1),
-          cachedOffset, offset, (cachedOffset - 1), cachedOffset, (offset
-              + count - 1)));
-
-      if (!pendingWrites.isEmpty()) {
-        LOG.warn("There are other pending writes, fail this jumbo write");
-        return null;
-      }
+      LOG.warn(String.format("Got overwrite with appended data [%d-%d),"
+          + " current offset %d," + " drop the overlapped section [%d-%d)"
+          + " and append new data [%d-%d).", offset, (offset + count),
+          cachedOffset, offset, cachedOffset, cachedOffset, (offset
+              + count)));
       
       LOG.warn("Modify this write to write only the appended data");
       alterWriteRequest(request, cachedOffset);
@@ -1002,45 +1022,56 @@ class OpenFileCtx {
       this.asyncStatus = false;
       return null;
     } 
-    
-      Entry<OffsetRange, WriteCtx> lastEntry = pendingWrites.lastEntry();
-      OffsetRange range = lastEntry.getKey();
-      WriteCtx toWrite = lastEntry.getValue();
-      
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("range.getMin()=" + range.getMin() + " nextOffset="
-            + nextOffset);
+
+    Entry<OffsetRange, WriteCtx> lastEntry = pendingWrites.lastEntry();
+    OffsetRange range = lastEntry.getKey();
+    WriteCtx toWrite = lastEntry.getValue();
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("range.getMin()=" + range.getMin() + " nextOffset="
+          + nextOffset);
+    }
+
+    long offset = nextOffset.get();
+    if (range.getMin() > offset) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("The next sequential write has not arrived yet");
       }
-      
-      long offset = nextOffset.get();
-      if (range.getMin() > offset) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("The next sequential write has not arrived yet");
-        }
-        processCommits(nextOffset.get()); // handle race
-        this.asyncStatus = false;
-      } else if (range.getMin() < offset && range.getMax() > offset) {
-        // shouldn't happen since we do sync for overlapped concurrent writers
-        LOG.warn("Got an overlapping write (" + range.getMin() + ", "
-            + range.getMax() + "), nextOffset=" + offset
-            + ". Silently drop it now");
-        pendingWrites.remove(range);
-        processCommits(nextOffset.get()); // handle race
-      } else {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Remove write(" + range.getMin() + "-" + range.getMax()
-              + ") from the list");
-        }
-        // after writing, remove the WriteCtx from cache 
-        pendingWrites.remove(range);
-        // update nextOffset
-        nextOffset.addAndGet(toWrite.getCount());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Change nextOffset to " + nextOffset.get());
-        }
-        return toWrite;
+      processCommits(nextOffset.get()); // handle race
+      this.asyncStatus = false;
+    } else if (range.getMax() <= offset) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Remove write " + range.toString()
+            + " which is already written from the list");
       }
-    
+      // remove the WriteCtx from cache
+      pendingWrites.remove(range);
+    } else if (range.getMin() < offset && range.getMax() > offset) {
+      LOG.warn("Got an overlapping write " + range.toString()
+          + ", nextOffset=" + offset
+          + ". Remove and trim it");
+      pendingWrites.remove(range);
+      trimWriteRequest(toWrite, offset);
+      // update nextOffset
+      nextOffset.addAndGet(toWrite.getCount());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Change nextOffset (after trim) to " + nextOffset.get());
+      }
+      return toWrite;
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Remove write " + range.toString()
+            + " from the list");
+      }
+      // after writing, remove the WriteCtx from cache
+      pendingWrites.remove(range);
+      // update nextOffset
+      nextOffset.addAndGet(toWrite.getCount());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Change nextOffset to " + nextOffset.get());
+      }
+      return toWrite;
+    }
     return null;
   }
   
@@ -1272,8 +1303,8 @@ class OpenFileCtx {
     WccAttr preOpAttr = latestAttr.getWccAttr();
     while (!pendingWrites.isEmpty()) {
       OffsetRange key = pendingWrites.firstKey();
-      LOG.info("Fail pending write: (" + key.getMin() + ", " + key.getMax()
-          + "), nextOffset=" + nextOffset.get());
+      LOG.info("Fail pending write: " + key.toString()
+          + ", nextOffset=" + nextOffset.get());
       
       WriteCtx writeCtx = pendingWrites.remove(key);
       if (!writeCtx.getReplied()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/151fca50/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
index 82c826f..8c2c7ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
@@ -51,8 +51,8 @@ class WriteCtx {
   }
 
   private final FileHandle handle;
-  private final long offset;
-  private final int count;
+  private long offset;
+  private int count;
   
   /**
    * Some clients can send a write that includes previously written data along
@@ -61,13 +61,61 @@ class WriteCtx {
    * request before it was modified to write only the new data. 
    * @see OpenFileCtx#addWritesToCache for more details
    */
-  private final int originalCount; 
+  private int originalCount;
   public static final int INVALID_ORIGINAL_COUNT = -1;
   
+  /**
+   * Overlapping Write Request Handling
+   * A write request can be in three states:
+   *   s0. just created, with data != null
+   *   s1. dumped as length "count", and data set to null
+   *   s2. read back from dumped area as length "count"
+   *
+   * Write requests may have overlapping range, we detect this by comparing
+   * the data offset range of the request against the current offset of data
+   * already written to HDFS. There are two categories:
+   *
+   * 1. If the beginning part of a new write request data is already written
+   * due to an earlier request, we alter the new request by trimming this
+   * portion before the new request enters state s0, and the originalCount is
+   * remembered.
+   *
+   * 2. If the lower end of the write request range is beyond the current
+   * offset of data already written, we put the request into cache, and detect
+   * the overlapping when taking the request out from cache.
+   *
+   * For category 2, if we find out that a write request overlap with another,
+   * this write request is already in state s0, s1, or s3. We trim the
+   * beginning part of this request, by remembering the size of this portion
+   * as trimDelta. So the resulted offset of the write request is
+   * "offset + trimDelta" and the resulted size of the write request is
+   * "count - trimDelta".
+   *
+   * What important to notice is, if the request is in s1 when we do the
+   * trimming, the data dumped is of size "count", so when we load
+   * the data back from dumped area, we should set the position of the data
+   * buffer to trimDelta.
+   */
+  private int trimDelta;
+
   public int getOriginalCount() {
     return originalCount;
   }
 
+  public void trimWrite(int delta) {
+    Preconditions.checkState(delta < count);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Trim write request by delta:" + delta + " " + toString());
+    }
+    synchronized(this) {
+      trimDelta = delta;
+      if (originalCount == INVALID_ORIGINAL_COUNT) {
+        originalCount = count;
+      }
+      trimData();
+    }
+  }
+
   private final WriteStableHow stableHow;
   private volatile ByteBuffer data;
   
@@ -139,11 +187,17 @@ class WriteCtx {
   }
   
   long getOffset() {
-    return offset;
+    synchronized(this) {
+      // See comment "Overlapping Write Request Handling" above
+      return offset + trimDelta;
+    }
   }
 
   int getCount() {
-    return count;
+    synchronized(this) {
+      // See comment "Overlapping Write Request Handling" above
+      return count - trimDelta;
+    }
   }
 
   WriteStableHow getStableHow() {
@@ -174,7 +228,22 @@ class WriteCtx {
       throw new IOException("Data count is " + count + ", but read back "
           + size + "bytes");
     }
-    data = ByteBuffer.wrap(rawData);
+    synchronized(this) {
+      data = ByteBuffer.wrap(rawData);
+      trimData();
+    }
+  }
+
+  private void trimData() {
+    if (data != null && trimDelta > 0) {
+      // make it not dump-able since the data  will be used
+      // shortly
+      dataState = DataState.NO_DUMP;
+      data.position(data.position() + trimDelta);
+      offset += trimDelta;
+      count -= trimDelta;
+      trimDelta = 0;
+    }
   }
 
   public void writeData(HdfsDataOutputStream fos) throws IOException {
@@ -229,6 +298,7 @@ class WriteCtx {
     this.offset = offset;
     this.count = count;
     this.originalCount = originalCount;
+    this.trimDelta = 0;
     this.stableHow = stableHow;
     this.data = data;
     this.channel = channel;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/151fca50/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
index 3c193ae..9c327c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
@@ -640,7 +640,97 @@ public class TestWrites {
       }
     }
   }
-  
+
+  @Test
+  public void testOverlappingWrites() throws IOException, InterruptedException {
+    NfsConfiguration config = new NfsConfiguration();
+    MiniDFSCluster cluster = null;
+    RpcProgramNfs3 nfsd;
+    final int bufSize = 32;
+    SecurityHandler securityHandler = Mockito.mock(SecurityHandler.class);
+    Mockito.when(securityHandler.getUser()).thenReturn(
+        System.getProperty("user.name"));
+    String currentUser = System.getProperty("user.name");
+    config.set(
+        DefaultImpersonationProvider.getTestProvider().
+            getProxySuperuserGroupConfKey(currentUser),
+        "*");
+    config.set(
+        DefaultImpersonationProvider.getTestProvider().
+            getProxySuperuserIpConfKey(currentUser),
+        "*");
+    ProxyUsers.refreshSuperUserGroupsConfiguration(config);
+    // Use emphral port in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+
+    try {
+      cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
+      cluster.waitActive();
+
+      Nfs3 nfs3 = new Nfs3(config);
+      nfs3.startServiceInternal(false);
+      nfsd = (RpcProgramNfs3) nfs3.getRpcProgram();
+
+      DFSClient dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config),
+          config);
+      HdfsFileStatus status = dfsClient.getFileInfo("/");
+      FileHandle rootHandle = new FileHandle(status.getFileId());
+
+      CREATE3Request createReq = new CREATE3Request(rootHandle,
+          "overlapping-writes" + System.currentTimeMillis(),
+          Nfs3Constant.CREATE_UNCHECKED, new SetAttr3(), 0);
+      XDR createXdr = new XDR();
+      createReq.serialize(createXdr);
+      CREATE3Response createRsp = nfsd.create(createXdr.asReadOnlyWrap(),
+          securityHandler, new InetSocketAddress("localhost", 1234));
+      FileHandle handle = createRsp.getObjHandle();
+      byte[] buffer = new byte[bufSize];
+      for (int i = 0; i < bufSize; i++) {
+        buffer[i] = (byte) i;
+      }
+      int[][] ranges = new int[][] {
+          {0, 10},
+          {5, 7},
+          {5, 5},
+          {10, 6},
+          {18, 6},
+          {20, 6},
+          {28, 4},
+          {16, 2},
+          {25, 4}
+      };
+      for (int i = 0; i < ranges.length; i++) {
+        int x[] = ranges[i];
+        byte[] tbuffer = new byte[x[1]];
+        for (int j = 0; j < x[1]; j++) {
+          tbuffer[j] = buffer[x[0] + j];
+        }
+        WRITE3Request writeReq = new WRITE3Request(handle, (long)x[0], x[1],
+            WriteStableHow.UNSTABLE, ByteBuffer.wrap(tbuffer));
+        XDR writeXdr = new XDR();
+        writeReq.serialize(writeXdr);
+        nfsd.write(writeXdr.asReadOnlyWrap(), null, 1, securityHandler,
+            new InetSocketAddress("localhost", 1234));
+      }
+
+      waitWrite(nfsd, handle, 60000);
+      READ3Request readReq = new READ3Request(handle, 0, bufSize);
+      XDR readXdr = new XDR();
+      readReq.serialize(readXdr);
+      READ3Response readRsp = nfsd.read(readXdr.asReadOnlyWrap(),
+          securityHandler, new InetSocketAddress("localhost", config.getInt(
+              NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY,
+              NfsConfigKeys.DFS_NFS_SERVER_PORT_DEFAULT)));
+
+      assertTrue(Arrays.equals(buffer, readRsp.getData().array()));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   @Test
   public void testCheckSequential() throws IOException {
     DFSClient dfsClient = Mockito.mock(DFSClient.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/151fca50/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 3daf8d4..d55beae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1447,6 +1447,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9147. Fix the setting of visibleLength in ExternalBlockReader. (Colin
     P. McCabe via Lei (Eddy) Xu)
 
+    HDFS-9092. Nfs silently drops overlapping write requests and causes data
+    copying to fail. (Yongjun Zhang)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[31/58] [abbrv] hadoop git commit: HDFS-8859. Improve DataNode ReplicaMap memory footprint to save about 45%. (yliu)

Posted by zh...@apache.org.
HDFS-8859. Improve DataNode ReplicaMap memory footprint to save about 45%. (yliu)


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

Branch: refs/heads/HDFS-7285
Commit: d6fa34e014b0e2a61b24f05dd08ebe12354267fd
Parents: 151fca5
Author: yliu <yl...@apache.org>
Authored: Tue Sep 29 16:20:35 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Tue Sep 29 16:20:35 2015 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/util/GSet.java  |  14 ++
 .../org/apache/hadoop/util/GSetByHashMap.java   |   6 +
 .../org/apache/hadoop/util/LightWeightGSet.java |  82 ++++--
 .../hadoop/util/LightWeightResizableGSet.java   | 129 ++++++++++
 .../java/org/apache/hadoop/util/TestGSet.java   |  69 ++++-
 .../hadoop/util/TestLightWeightCache.java       |   6 +
 .../util/TestLightWeightResizableGSet.java      | 252 +++++++++++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/datanode/ReplicaInfo.java       |  27 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |   7 +-
 .../datanode/fsdataset/impl/ReplicaMap.java     |  38 +--
 11 files changed, 569 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSet.java
index 26e73cf..e4a8d0f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSet.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSet.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.util;
 
+import java.util.Collection;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -86,5 +88,17 @@ public interface GSet<K, E extends K> extends Iterable<E> {
   */
   E remove(K key);
 
+  /**
+   * Clear the set.
+   */
   void clear();
+
+  /**
+   * Returns a {@link Collection} view of the values contained in this set.
+   * The collection is backed by the set, so changes to the set are
+   * reflected in the collection, and vice-versa.
+   *
+   * @return the collection of values.
+   */
+  Collection<E> values();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSetByHashMap.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSetByHashMap.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSetByHashMap.java
index 87488db..e341c74 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSetByHashMap.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GSetByHashMap.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.util;
 
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 
@@ -70,4 +71,9 @@ public class GSetByHashMap<K, E extends K> implements GSet<K, E> {
   public void clear() {
     m.clear();
   }
+
+  @Override
+  public Collection<E> values() {
+    return m.values();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
index 1767d85..7c7878a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
@@ -18,12 +18,14 @@
 package org.apache.hadoop.util;
 
 import java.io.PrintStream;
+import java.util.AbstractCollection;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.Iterator;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -49,12 +51,12 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
   /**
    * Elements of {@link LightWeightGSet}.
    */
-  public static interface LinkedElement {
+  public interface LinkedElement {
     /** Set the next element. */
-    public void setNext(LinkedElement next);
+    void setNext(LinkedElement next);
 
     /** Get the next element. */
-    public LinkedElement getNext();
+    LinkedElement getNext();
   }
 
   static final int MAX_ARRAY_LENGTH = 1 << 30; //prevent int overflow problem
@@ -64,15 +66,20 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
    * An internal array of entries, which are the rows of the hash table.
    * The size must be a power of two.
    */
-  private final LinkedElement[] entries;
+  protected LinkedElement[] entries;
   /** A mask for computing the array index from the hash value of an element. */
-  private final int hash_mask;
+  protected int hash_mask;
   /** The size of the set (not the entry array). */
-  private int size = 0;
+  protected int size = 0;
   /** Modification version for fail-fast.
    * @see ConcurrentModificationException
    */
-  private int modification = 0;
+  protected int modification = 0;
+
+  private Collection<E> values;
+
+  protected LightWeightGSet() {
+  }
 
   /**
    * @param recommended_length Recommended size of the internal array.
@@ -87,7 +94,7 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
   }
 
   //compute actual length
-  private static int actualArrayLength(int recommended) {
+  protected static int actualArrayLength(int recommended) {
     if (recommended > MAX_ARRAY_LENGTH) {
       return MAX_ARRAY_LENGTH;
     } else if (recommended < MIN_ARRAY_LENGTH) {
@@ -103,11 +110,11 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
     return size;
   }
 
-  private int getIndex(final K key) {
+  protected int getIndex(final K key) {
     return key.hashCode() & hash_mask;
   }
 
-  private E convert(final LinkedElement e){
+  protected E convert(final LinkedElement e){
     @SuppressWarnings("unchecked")
     final E r = (E)e;
     return r;
@@ -138,24 +145,26 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
 
   @Override
   public E put(final E element) {
-    //validate element
+    // validate element
     if (element == null) {
       throw new NullPointerException("Null element is not supported.");
     }
-    if (!(element instanceof LinkedElement)) {
+    LinkedElement e = null;
+    try {
+      e = (LinkedElement)element;
+    } catch (ClassCastException ex) {
       throw new HadoopIllegalArgumentException(
           "!(element instanceof LinkedElement), element.getClass()="
           + element.getClass());
     }
-    final LinkedElement e = (LinkedElement)element;
 
-    //find index
+    // find index
     final int index = getIndex(element);
 
-    //remove if it already exists
+    // remove if it already exists
     final E existing = remove(index, element);
 
-    //insert the element to the head of the linked list
+    // insert the element to the head of the linked list
     modification++;
     size++;
     e.setNext(entries[index]);
@@ -171,7 +180,7 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
    * @return If such element exists, return it.
    *         Otherwise, return null.
    */
-  private E remove(final int index, final K key) {
+  protected E remove(final int index, final K key) {
     if (entries[index] == null) {
       return null;
     } else if (entries[index].equals(key)) {
@@ -214,6 +223,38 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
   }
 
   @Override
+  public Collection<E> values() {
+    if (values == null) {
+      values = new Values();
+    }
+    return values;
+  }
+
+  private final class Values extends AbstractCollection<E> {
+
+    @Override
+    public Iterator<E> iterator() {
+      return LightWeightGSet.this.iterator();
+    }
+
+    @Override
+    public int size() {
+      return size;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean contains(Object o) {
+      return LightWeightGSet.this.contains((K)o);
+    }
+
+    @Override
+    public void clear() {
+      LightWeightGSet.this.clear();
+    }
+  }
+
+  @Override
   public Iterator<E> iterator() {
     return new SetIterator();
   }
@@ -363,9 +404,8 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
   }
   
   public void clear() {
-    for (int i = 0; i < entries.length; i++) {
-      entries[i] = null;
-    }
+    modification++;
+    Arrays.fill(entries, null);
     size = 0;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java
new file mode 100644
index 0000000..0abcf98
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java
@@ -0,0 +1,129 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A low memory footprint {@link GSet} implementation,
+ * which uses an array for storing the elements
+ * and linked lists for collision resolution.
+ *
+ * If the size of elements exceeds the threshold,
+ * the internal array will be resized to double length.
+ *
+ * This class does not support null element.
+ *
+ * This class is not thread safe.
+ *
+ * @param <K> Key type for looking up the elements
+ * @param <E> Element type, which must be
+ *       (1) a subclass of K, and
+ *       (2) implementing {@link LinkedElement} interface.
+ */
+@InterfaceAudience.Private
+public class LightWeightResizableGSet<K, E extends K>
+    extends LightWeightGSet<K, E> {
+
+  /**
+   * The default initial capacity - MUST be a power of two.
+   */
+  static final int DEFAULT_INITIAL_CAPACITY = 1 << 4;
+
+  /**
+   * The load factor used when none specified in constructor.
+   */
+  static final float DEFAULT_LOAD_FACTOR = 0.75f;
+
+  /** Size of the entry table. */
+  private int capacity;
+
+  /**
+   * The load factor for the hash set.
+   */
+  private final float loadFactor;
+  private int threshold;
+
+  public LightWeightResizableGSet(int initCapacity, float loadFactor) {
+    if (initCapacity < 0) {
+      throw new HadoopIllegalArgumentException("Illegal initial capacity: " +
+          initCapacity);
+    }
+    if (loadFactor <= 0 || loadFactor > 1.0f) {
+      throw new HadoopIllegalArgumentException("Illegal load factor: " +
+          loadFactor);
+    }
+    this.capacity = actualArrayLength(initCapacity);
+    this.hash_mask = capacity - 1;
+    this.loadFactor = loadFactor;
+    this.threshold = (int) (capacity * loadFactor);
+
+    entries = new LinkedElement[capacity];
+  }
+
+  public LightWeightResizableGSet() {
+    this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR);
+  }
+
+  public LightWeightResizableGSet(int initCapacity) {
+    this(initCapacity, DEFAULT_LOAD_FACTOR);
+  }
+
+  @Override
+  public E put(final E element) {
+    E existing = super.put(element);
+    expandIfNecessary();
+    return existing;
+  }
+
+  /**
+   * Resize the internal table to given capacity.
+   */
+  @SuppressWarnings("unchecked")
+  protected void resize(int cap) {
+    int newCapacity = actualArrayLength(cap);
+    if (newCapacity == this.capacity) {
+      return;
+    }
+    this.capacity = newCapacity;
+    this.threshold = (int) (capacity * loadFactor);
+    this.hash_mask = capacity - 1;
+    LinkedElement[] oldEntries = entries;
+    entries = new LinkedElement[capacity];
+    for (int i = 0; i < oldEntries.length; i++) {
+      LinkedElement e = oldEntries[i];
+      while (e != null) {
+        LinkedElement next = e.getNext();
+        int index = getIndex((E)e);
+        e.setNext(entries[index]);
+        entries[index] = e;
+        e = next;
+      }
+    }
+  }
+
+  /**
+   * Checks if we need to expand, and expands if necessary.
+   */
+  protected void expandIfNecessary() {
+    if (size > this.threshold && capacity < MAX_ARRAY_LENGTH) {
+      resize(capacity * 2);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGSet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGSet.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGSet.java
index af880ee..2d39f3d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGSet.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGSet.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.util;
 
+import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.Iterator;
 import java.util.Random;
@@ -41,10 +42,15 @@ public class TestGSet {
 
   @Test
   public void testExceptionCases() {
+    testExceptionCases(false);
+    testExceptionCases(true);
+  }
+
+  private void testExceptionCases(boolean resizable) {
     {
       //test contains
       final LightWeightGSet<Integer, Integer> gset
-        = new LightWeightGSet<Integer, Integer>(16);
+        = createGSet(16, resizable);
       try {
         //test contains with a null element
         gset.contains(null);
@@ -57,7 +63,7 @@ public class TestGSet {
     {
       //test get
       final LightWeightGSet<Integer, Integer> gset
-        = new LightWeightGSet<Integer, Integer>(16);
+        = createGSet(16, resizable);
       try {
         //test get with a null element
         gset.get(null);
@@ -70,7 +76,7 @@ public class TestGSet {
     {
       //test put
       final LightWeightGSet<Integer, Integer> gset
-        = new LightWeightGSet<Integer, Integer>(16);
+        = createGSet(16, resizable);
       try {
         //test put with a null element
         gset.put(null);
@@ -97,7 +103,7 @@ public class TestGSet {
       for(int v = 1; v < data.length-1; v++) {
         {
           //test remove while iterating
-          final GSet<IntElement, IntElement> gset = createGSet(data);
+          final GSet<IntElement, IntElement> gset = createGSet(data, resizable);
           for(IntElement i : gset) {
             if (i.value == v) {
               //okay because data[0] is not in gset
@@ -120,7 +126,7 @@ public class TestGSet {
 
         {
           //test put new element while iterating
-          final GSet<IntElement, IntElement> gset = createGSet(data);
+          final GSet<IntElement, IntElement> gset = createGSet(data, resizable);
           try {
             for(IntElement i : gset) {
               if (i.value == v) {
@@ -135,7 +141,7 @@ public class TestGSet {
 
         {
           //test put existing element while iterating
-          final GSet<IntElement, IntElement> gset = createGSet(data);
+          final GSet<IntElement, IntElement> gset = createGSet(data, resizable);
           try {
             for(IntElement i : gset) {
               if (i.value == v) {
@@ -151,9 +157,17 @@ public class TestGSet {
     }
   }
 
-  private static GSet<IntElement, IntElement> createGSet(final IntElement[] data) {
+  private static LightWeightGSet<Integer, Integer> createGSet(
+      int size, boolean resizable) {
+    return resizable ? new LightWeightResizableGSet<Integer, Integer>(size) :
+      new LightWeightGSet<Integer, Integer>(size);
+  }
+
+  private static GSet<IntElement, IntElement> createGSet(
+      final IntElement[] data, boolean resizable) {
     final GSet<IntElement, IntElement> gset
-      = new LightWeightGSet<IntElement, IntElement>(8);
+      = resizable ? new LightWeightResizableGSet<IntElement, IntElement>(8) :
+        new LightWeightGSet<IntElement, IntElement>(8);
     for(int i = 1; i < data.length; i++) {
       gset.put(data[i]);
     }
@@ -168,6 +182,14 @@ public class TestGSet {
     check(new GSetTestCase(255, 1 << 10, 65537));
   }
 
+  @Test
+  public void testResizableGSet() {
+    //The parameters are: table length, data size, modulus, resizable.
+    check(new GSetTestCase(1, 1 << 4, 65537, true));
+    check(new GSetTestCase(17, 1 << 16, 17, true));
+    check(new GSetTestCase(255, 1 << 10, 65537, true));
+  }
+
   /**
    * A long running test with various data sets and parameters.
    * It may take ~5 hours, 
@@ -177,14 +199,25 @@ public class TestGSet {
   //@Test
   public void runMultipleTestGSet() {
     for(int offset = -2; offset <= 2; offset++) {
-      runTestGSet(1, offset);
+      runTestGSet(1, offset, false);
+      for(int i = 1; i < Integer.SIZE - 1; i++) {
+        runTestGSet((1 << i) + 1, offset, false);
+      }
+    }
+  }
+
+  //@Test
+  public void runMultipleTestResizableGSet() {
+    for(int offset = -2; offset <= 2; offset++) {
+      runTestGSet(1, offset, true);
       for(int i = 1; i < Integer.SIZE - 1; i++) {
-        runTestGSet((1 << i) + 1, offset);
+        runTestGSet((1 << i) + 1, offset, true);
       }
     }
   }
 
-  private static void runTestGSet(final int modulus, final int offset) {
+  private static void runTestGSet(final int modulus, final int offset,
+      boolean resizable) {
     println("\n\nmodulus=" + modulus + ", offset=" + offset);
     for(int i = 0; i <= 16; i += 4) {
       final int tablelength = (1 << i) + offset;
@@ -194,7 +227,7 @@ public class TestGSet {
 
       for(int j = 0; j <= upper; j += steps) {
         final int datasize = 1 << j;
-        check(new GSetTestCase(tablelength, datasize, modulus));
+        check(new GSetTestCase(tablelength, datasize, modulus, resizable));
       }
     }
   }
@@ -265,6 +298,10 @@ public class TestGSet {
     int contain_count = 0;
 
     GSetTestCase(int tablelength, int datasize, int modulus) {
+      this(tablelength, datasize, modulus, false);
+    }
+
+    GSetTestCase(int tablelength, int datasize, int modulus, boolean resizable) {
       denominator = Math.min((datasize >> 7) + 1, 1 << 16);
       info = getClass().getSimpleName()
           + ": tablelength=" + tablelength
@@ -274,7 +311,8 @@ public class TestGSet {
       println(info);
 
       data  = new IntData(datasize, modulus);
-      gset = new LightWeightGSet<IntElement, IntElement>(tablelength);
+      gset = resizable ? new LightWeightResizableGSet<IntElement, IntElement>() :
+        new LightWeightGSet<IntElement, IntElement>(tablelength);
 
       Assert.assertEquals(0, gset.size());
     }
@@ -392,6 +430,11 @@ public class TestGSet {
       gset.clear();
       Assert.assertEquals(0, size());
     }
+
+    @Override
+    public Collection<IntElement> values() {
+      throw new UnsupportedOperationException();
+    }
   }
 
   /** Test data set */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightCache.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightCache.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightCache.java
index 68d484f..dff6937 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightCache.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightCache.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.util;
 
+import java.util.Collection;
 import java.util.Date;
 import java.util.Iterator;
 import java.util.Random;
@@ -379,6 +380,11 @@ public class TestLightWeightCache {
       cache.clear();
       Assert.assertEquals(0, size());
     }
+
+    @Override
+    public Collection<IntEntry> values() {
+      throw new UnsupportedOperationException();
+    }
   }
 
   private static class IntData {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightResizableGSet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightResizableGSet.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightResizableGSet.java
new file mode 100644
index 0000000..3250092
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightResizableGSet.java
@@ -0,0 +1,252 @@
+/**
+ * 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.util;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/** Testing {@link LightWeightResizableGSet} */
+public class TestLightWeightResizableGSet {
+  public static final Log LOG = LogFactory.getLog(TestLightWeightResizableGSet.class);
+  private Random random = new Random();
+
+  private TestElement[] generateElements(int length) {
+    TestElement[] elements = new TestElement[length];
+    Set<Long> keys = new HashSet<>();
+    long k = 0;
+    for (int i = 0; i < length; i++) {
+      while (keys.contains(k = random.nextLong()));
+      elements[i] = new TestElement(k, random.nextLong());
+      keys.add(k);
+    }
+    return elements;
+  }
+
+  private TestKey[] getKeys(TestElement[] elements) {
+    TestKey[] keys = new TestKey[elements.length];
+    for (int i = 0; i < elements.length; i++) {
+      keys[i] = new TestKey(elements[i].getKey());
+    }
+    return keys;
+  }
+
+  private TestElement[] generateElements(TestKey[] keys) {
+    TestElement[] elements = new TestElement[keys.length];
+    for (int i = 0; i < keys.length; i++) {
+      elements[i] = new TestElement(keys[i], random.nextLong());
+    }
+    return elements;
+  }
+
+  private static class TestKey {
+    private final long key;
+
+    TestKey(long key) {
+      this.key = key;
+    }
+
+    TestKey(TestKey other) {
+      this.key = other.key;
+    }
+
+    long getKey() {
+      return key;
+    }
+
+    @Override
+    public int hashCode() {
+      return (int)(key^(key>>>32));
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof TestKey)) {
+        return false;
+      }
+      TestKey other = (TestKey)o;
+      return key == other.key;
+    }
+  }
+
+  private static class TestElement extends TestKey
+      implements LightWeightResizableGSet.LinkedElement {
+    private final long data;
+    private LightWeightResizableGSet.LinkedElement next;
+
+    TestElement(long key, long data) {
+      super(key);
+      this.data = data;
+    }
+
+    TestElement(TestKey key, long data) {
+      super(key);
+      this.data = data;
+    }
+
+    long getData() {
+      return data;
+    }
+
+    @Override
+    public void setNext(LightWeightResizableGSet.LinkedElement next) {
+      this.next = next;
+    }
+
+    @Override
+    public LightWeightResizableGSet.LinkedElement getNext() {
+      return next;
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testBasicOperations() {
+    TestElement[] elements = generateElements(1 << 16);
+    final LightWeightResizableGSet<TestKey, TestElement> set =
+        new LightWeightResizableGSet<TestKey, TestElement>();
+
+    assertEquals(set.size(), 0);
+
+    // put all elements
+    for (int i = 0; i < elements.length; i++) {
+      TestElement element = set.put(elements[i]);
+      assertTrue(element == null);
+    }
+
+    // check the set size
+    assertEquals(set.size(), elements.length);
+
+    // check all elements exist in the set and the data is correct
+    for (int i = 0; i < elements.length; i++) {
+      assertTrue(set.contains(elements[i]));
+
+      TestElement element = set.get(elements[i]);
+      assertEquals(elements[i].getData(), element.getData());
+    }
+
+    TestKey[] keys = getKeys(elements);
+    // generate new elements with same key, but new data
+    TestElement[] newElements = generateElements(keys);
+    // update the set
+    for (int i = 0; i < newElements.length; i++) {
+      TestElement element = set.put(newElements[i]);
+      assertTrue(element != null);
+    }
+
+    // check the set size
+    assertEquals(set.size(), elements.length);
+
+    // check all elements exist in the set and the data is updated to new value
+    for (int i = 0; i < keys.length; i++) {
+      assertTrue(set.contains(keys[i]));
+
+      TestElement element = set.get(keys[i]);
+      assertEquals(newElements[i].getData(), element.getData());
+    }
+
+    // test LightWeightHashGSet#values
+    Collection<TestElement> cElements = set.values();
+    assertEquals(cElements.size(), elements.length);
+    for (TestElement element : cElements) {
+      assertTrue(set.contains(element));
+    }
+
+    // remove elements
+    for (int i = 0; i < keys.length; i++) {
+      TestElement element = set.remove(keys[i]);
+
+      assertTrue(element != null);
+
+      // the element should not exist after remove
+      assertFalse(set.contains(keys[i]));
+    }
+
+    // check the set size
+    assertEquals(set.size(), 0);
+  }
+
+  @Test(timeout = 60000)
+  public void testRemoveAll() {
+    TestElement[] elements = generateElements(1 << 16);
+    final LightWeightResizableGSet<TestKey, TestElement> set =
+        new LightWeightResizableGSet<TestKey, TestElement>();
+
+    assertEquals(set.size(), 0);
+
+    // put all elements
+    for (int i = 0; i < elements.length; i++) {
+      TestElement element = set.put(elements[i]);
+      assertTrue(element == null);
+    }
+
+    // check the set size
+    assertEquals(set.size(), elements.length);
+
+    // remove all through clear
+    {
+      set.clear();
+      assertEquals(set.size(), 0);
+
+      // check all elements removed
+      for (int i = 0; i < elements.length; i++) {
+        assertFalse(set.contains(elements[i]));
+      }
+      assertFalse(set.iterator().hasNext());
+    }
+
+    // put all elements back
+    for (int i = 0; i < elements.length; i++) {
+      TestElement element = set.put(elements[i]);
+      assertTrue(element == null);
+    }
+
+    // remove all through iterator
+    {
+      for (Iterator<TestElement> iter = set.iterator(); iter.hasNext(); ) {
+        TestElement element = iter.next();
+        // element should be there before removing
+        assertTrue(set.contains(element));
+        iter.remove();
+        // element should not be there now
+        assertFalse(set.contains(element));
+      }
+
+      // the deleted elements should not be there
+      for (int i = 0; i < elements.length; i++) {
+        assertFalse(set.contains(elements[i]));
+      }
+
+      // iterator should not have next
+      assertFalse(set.iterator().hasNext());
+
+      // check the set size
+      assertEquals(set.size(), 0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/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 d55beae..182464b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -991,6 +991,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9148. Incorrect assert message in TestWriteToReplica#testWriteToTemporary
     (Tony Wu via lei)
 
+    HDFS-8859. Improve DataNode ReplicaMap memory footprint to save about 45%.
+    (yliu)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
index 31b14fa..d19e656 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
@@ -18,20 +18,13 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.LightWeightResizableGSet;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -40,8 +33,12 @@ import com.google.common.annotations.VisibleForTesting;
  * It provides a general interface for meta information of a replica.
  */
 @InterfaceAudience.Private
-abstract public class ReplicaInfo extends Block implements Replica {
-  
+abstract public class ReplicaInfo extends Block
+    implements Replica, LightWeightResizableGSet.LinkedElement {
+
+  /** For implementing {@link LightWeightResizableGSet.LinkedElement} interface */
+  private LightWeightResizableGSet.LinkedElement next;
+
   /** volume where the replica belongs */
   private FsVolumeSpi volume;
   
@@ -229,4 +226,14 @@ abstract public class ReplicaInfo extends Block implements Replica {
   public boolean isOnTransientStorage() {
     return volume.isTransientStorage();
   }
+
+  @Override
+  public LightWeightResizableGSet.LinkedElement getNext() {
+    return next;
+  }
+
+  @Override
+  public void setNext(LightWeightResizableGSet.LinkedElement next) {
+    this.next = next;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/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 68c951a..571f085 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
@@ -743,7 +743,12 @@ class BlockPoolSlice {
       // Now it is safe to add the replica into volumeMap
       // In case of any exception during parsing this cache file, fall back
       // to scan all the files on disk.
-      for (ReplicaInfo info: tmpReplicaMap.replicas(bpid)) {
+      for (Iterator<ReplicaInfo> iter =
+          tmpReplicaMap.replicas(bpid).iterator(); iter.hasNext(); ) {
+        ReplicaInfo info = iter.next();
+        // We use a lightweight GSet to store replicaInfo, we need to remove
+        // it from one GSet before adding to another.
+        iter.remove();
         volumeMap.add(bpid, info);
       }
       LOG.info("Successfully read replica from cache file : " 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6fa34e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
index 617e0fd..6f0b8a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.util.LightWeightResizableGSet;
 
 /**
  * Maintains the replica map. 
@@ -33,9 +34,9 @@ class ReplicaMap {
   private final Object mutex;
   
   // Map of block pool Id to another map of block Id to ReplicaInfo.
-  private final Map<String, Map<Long, ReplicaInfo>> map =
-    new HashMap<String, Map<Long, ReplicaInfo>>();
-  
+  private final Map<String, LightWeightResizableGSet<Block, ReplicaInfo>> map =
+    new HashMap<String, LightWeightResizableGSet<Block, ReplicaInfo>>();
+
   ReplicaMap(Object mutex) {
     if (mutex == null) {
       throw new HadoopIllegalArgumentException(
@@ -91,8 +92,8 @@ class ReplicaMap {
   ReplicaInfo get(String bpid, long blockId) {
     checkBlockPool(bpid);
     synchronized(mutex) {
-      Map<Long, ReplicaInfo> m = map.get(bpid);
-      return m != null ? m.get(blockId) : null;
+      LightWeightResizableGSet<Block, ReplicaInfo> m = map.get(bpid);
+      return m != null ? m.get(new Block(blockId)) : null;
     }
   }
   
@@ -108,13 +109,13 @@ class ReplicaMap {
     checkBlockPool(bpid);
     checkBlock(replicaInfo);
     synchronized(mutex) {
-      Map<Long, ReplicaInfo> m = map.get(bpid);
+      LightWeightResizableGSet<Block, ReplicaInfo> m = map.get(bpid);
       if (m == null) {
         // Add an entry for block pool if it does not exist already
-        m = new HashMap<Long, ReplicaInfo>();
+        m = new LightWeightResizableGSet<Block, ReplicaInfo>();
         map.put(bpid, m);
       }
-      return  m.put(replicaInfo.getBlockId(), replicaInfo);
+      return  m.put(replicaInfo);
     }
   }
 
@@ -137,14 +138,13 @@ class ReplicaMap {
     checkBlockPool(bpid);
     checkBlock(block);
     synchronized(mutex) {
-      Map<Long, ReplicaInfo> m = map.get(bpid);
+      LightWeightResizableGSet<Block, ReplicaInfo> m = map.get(bpid);
       if (m != null) {
-        Long key = Long.valueOf(block.getBlockId());
-        ReplicaInfo replicaInfo = m.get(key);
+        ReplicaInfo replicaInfo = m.get(block);
         if (replicaInfo != null &&
             block.getGenerationStamp() == replicaInfo.getGenerationStamp()) {
-          return m.remove(key);
-        } 
+          return m.remove(block);
+        }
       }
     }
     
@@ -160,9 +160,9 @@ class ReplicaMap {
   ReplicaInfo remove(String bpid, long blockId) {
     checkBlockPool(bpid);
     synchronized(mutex) {
-      Map<Long, ReplicaInfo> m = map.get(bpid);
+      LightWeightResizableGSet<Block, ReplicaInfo> m = map.get(bpid);
       if (m != null) {
-        return m.remove(blockId);
+        return m.remove(new Block(blockId));
       }
     }
     return null;
@@ -174,7 +174,7 @@ class ReplicaMap {
    * @return the number of replicas in the map
    */
   int size(String bpid) {
-    Map<Long, ReplicaInfo> m = null;
+    LightWeightResizableGSet<Block, ReplicaInfo> m = null;
     synchronized(mutex) {
       m = map.get(bpid);
       return m != null ? m.size() : 0;
@@ -192,7 +192,7 @@ class ReplicaMap {
    * @return a collection of the replicas belonging to the block pool
    */
   Collection<ReplicaInfo> replicas(String bpid) {
-    Map<Long, ReplicaInfo> m = null;
+    LightWeightResizableGSet<Block, ReplicaInfo> m = null;
     m = map.get(bpid);
     return m != null ? m.values() : null;
   }
@@ -200,10 +200,10 @@ class ReplicaMap {
   void initBlockPool(String bpid) {
     checkBlockPool(bpid);
     synchronized(mutex) {
-      Map<Long, ReplicaInfo> m = map.get(bpid);
+      LightWeightResizableGSet<Block, ReplicaInfo> m = map.get(bpid);
       if (m == null) {
         // Add an entry for block pool if it does not exist already
-        m = new HashMap<Long, ReplicaInfo>();
+        m = new LightWeightResizableGSet<Block, ReplicaInfo>();
         map.put(bpid, m);
       }
     }


[55/58] [abbrv] hadoop git commit: HDFS-9001. DFSUtil.getNsServiceRpcUris() can return too many entries in a non-HA, non-federated cluster. Contributed by Daniel Templeton.

Posted by zh...@apache.org.
HDFS-9001. DFSUtil.getNsServiceRpcUris() can return too many entries in a non-HA, non-federated cluster. Contributed by Daniel Templeton.


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

Branch: refs/heads/HDFS-7285
Commit: 071733dc69a6f83c0cdca046b31ffd4f13304e93
Parents: 39285e6
Author: Aaron T. Myers <at...@apache.org>
Authored: Tue Sep 29 18:19:31 2015 -0700
Committer: Aaron T. Myers <at...@apache.org>
Committed: Tue Sep 29 18:19:31 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  37 +++--
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     | 144 +++++++++++++------
 3 files changed, 130 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/071733dc/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 dfd0b57..cedf1a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1468,6 +1468,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9174. Fix findbugs warnings in FSOutputSummer.tracer and
     DirectoryScanner$ReportCompiler.currentThread. (Yi Liu via wheat9)
 
+    HDFS-9001. DFSUtil.getNsServiceRpcUris() can return too many entries in a
+    non-HA, non-federated cluster. (Daniel Templeton via atm)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/071733dc/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 5b11ac2..5d405ab 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
@@ -732,20 +732,29 @@ public class DFSUtil {
         }
       }
     }
-    
-    // Add the default URI if it is an HDFS URI.
-    URI defaultUri = FileSystem.getDefaultUri(conf);
-    // checks if defaultUri is ip:port format
-    // and convert it to hostname:port format
-    if (defaultUri != null && (defaultUri.getPort() != -1)) {
-      defaultUri = createUri(defaultUri.getScheme(),
-          NetUtils.createSocketAddr(defaultUri.getHost(), 
-              defaultUri.getPort()));
-    }
-    if (defaultUri != null &&
-        HdfsConstants.HDFS_URI_SCHEME.equals(defaultUri.getScheme()) &&
-        !nonPreferredUris.contains(defaultUri)) {
-      ret.add(defaultUri);
+
+    // Add the default URI if it is an HDFS URI and we haven't come up with a
+    // valid non-nameservice NN address yet.  Consider the servicerpc-address
+    // and rpc-address to be the "unnamed" nameservice.  defaultFS is our
+    // fallback when rpc-address isn't given.  We therefore only want to add
+    // the defaultFS when neither the servicerpc-address (which is preferred)
+    // nor the rpc-address (which overrides defaultFS) is given.
+    if (!uriFound) {
+      URI defaultUri = FileSystem.getDefaultUri(conf);
+
+      // checks if defaultUri is ip:port format
+      // and convert it to hostname:port format
+      if (defaultUri != null && (defaultUri.getPort() != -1)) {
+        defaultUri = createUri(defaultUri.getScheme(),
+            NetUtils.createSocketAddr(defaultUri.getHost(),
+                defaultUri.getPort()));
+      }
+
+      if (defaultUri != null &&
+          HdfsConstants.HDFS_URI_SCHEME.equals(defaultUri.getScheme()) &&
+          !nonPreferredUris.contains(defaultUri)) {
+        ret.add(defaultUri);
+      }
     }
     
     return ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/071733dc/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 3435b7f..f22deaf 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
@@ -616,78 +616,142 @@ public class TestDFSUtil {
         DFSUtil.substituteForWildcardAddress("127.0.0.1:12345", "foo"));
   }
   
+  /**
+   * Test how name service URIs are handled with a variety of configuration
+   * settings
+   * @throws Exception
+   */
   @Test
   public void testGetNNUris() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
-    
+
     final String NS1_NN1_ADDR   = "ns1-nn1.example.com:8020";
     final String NS1_NN2_ADDR   = "ns1-nn2.example.com:8020";
     final String NS2_NN_ADDR    = "ns2-nn.example.com:8020";
     final String NN1_ADDR       = "nn.example.com:8020";
     final String NN1_SRVC_ADDR  = "nn.example.com:8021";
     final String NN2_ADDR       = "nn2.example.com:8020";
-    
+
+    conf.set(DFS_NAMESERVICES, "ns1");
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns1"), NS1_NN1_ADDR);
+
+    conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "hdfs://" + NN2_ADDR);
+    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR);
+
+    Collection<URI> uris = DFSUtil.getNameServiceUris(conf,
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,  DFS_NAMENODE_RPC_ADDRESS_KEY);
+
+    assertEquals("Incorrect number of URIs returned", 2, uris.size());
+    assertTrue("Missing URI for name service ns1",
+        uris.contains(new URI("hdfs://" + NS1_NN1_ADDR)));
+    assertTrue("Missing URI for service address",
+        uris.contains(new URI("hdfs://" + NN2_ADDR)));
+
+    conf = new HdfsConfiguration();
     conf.set(DFS_NAMESERVICES, "ns1,ns2");
-    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2");
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),
+        "nn1,nn2");
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_ADDR);
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn2"), NS1_NN2_ADDR);
-    
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns2"),
-        NS2_NN_ADDR);
-    
+
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns2"), NS2_NN_ADDR);
+
     conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "hdfs://" + NN1_ADDR);
-    
+
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN2_ADDR);
-    
-    Collection<URI> uris = DFSUtil.getNameServiceUris(conf,
+
+    uris = DFSUtil.getNameServiceUris(conf,
         DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,  DFS_NAMENODE_RPC_ADDRESS_KEY);
-    
-    assertEquals(4, uris.size());
-    assertTrue(uris.contains(new URI("hdfs://ns1")));
-    assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
-    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)));
-    assertTrue(uris.contains(new URI("hdfs://" + NN2_ADDR)));
-    
+
+    assertEquals("Incorrect number of URIs returned", 3, uris.size());
+    assertTrue("Missing URI for name service ns1",
+        uris.contains(new URI("hdfs://ns1")));
+    assertTrue("Missing URI for name service ns2",
+        uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
+    assertTrue("Missing URI for RPC address",
+        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+
     // Make sure that non-HDFS URIs in fs.defaultFS don't get included.
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
         "viewfs://vfs-name.example.com");
-    
-    uris = DFSUtil.getNameServiceUris(conf, DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, 
-        DFS_NAMENODE_RPC_ADDRESS_KEY);
-    
-    assertEquals(3, uris.size());
-    assertTrue(uris.contains(new URI("hdfs://ns1")));
-    assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
-    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)));
-    
+
+    uris = DFSUtil.getNameServiceUris(conf,
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
+
+    assertEquals("Incorrect number of URIs returned", 3, uris.size());
+    assertTrue("Missing URI for name service ns1",
+        uris.contains(new URI("hdfs://ns1")));
+    assertTrue("Missing URI for name service ns2",
+        uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
+    assertTrue("Missing URI for RPC address",
+        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+
     // Make sure that an HA URI being the default URI doesn't result in multiple
     // entries being returned.
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1");
     
-    uris = DFSUtil.getNameServiceUris(conf, DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, 
-        DFS_NAMENODE_RPC_ADDRESS_KEY);
-    
-    assertEquals(3, uris.size());
-    assertTrue(uris.contains(new URI("hdfs://ns1")));
-    assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
-    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)));
-    
+    uris = DFSUtil.getNameServiceUris(conf,
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
+
+    assertEquals("Incorrect number of URIs returned", 3, uris.size());
+    assertTrue("Missing URI for name service ns1",
+        uris.contains(new URI("hdfs://ns1")));
+    assertTrue("Missing URI for name service ns2",
+        uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
+    assertTrue("Missing URI for RPC address",
+        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+
+    // Check that the default URI is returned if there's nothing else to return.
+    conf = new HdfsConfiguration();
+    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR);
+
+    uris = DFSUtil.getNameServiceUris(conf,
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
+
+    assertEquals("Incorrect number of URIs returned", 1, uris.size());
+    assertTrue("Missing URI for RPC address (defaultFS)",
+        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+
+    // Check that the RPC address is the only address returned when the RPC
+    // and the default FS is given.
+    conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, NN2_ADDR);
+
+    uris = DFSUtil.getNameServiceUris(conf,
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
+
+    assertEquals("Incorrect number of URIs returned", 1, uris.size());
+    assertTrue("Missing URI for RPC address",
+        uris.contains(new URI("hdfs://" + NN2_ADDR)));
+
     // Make sure that when a service RPC address is used that is distinct from
     // the client RPC address, and that client RPC address is also used as the
     // default URI, that the client URI does not end up in the set of URIs
     // returned.
+    conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, NN1_ADDR);
+
+    uris = DFSUtil.getNameServiceUris(conf,
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
+
+    assertEquals("Incorrect number of URIs returned", 1, uris.size());
+    assertTrue("Missing URI for service ns1",
+        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+
+    // Check that when the default FS and service address are given, but
+    // the RPC address isn't, that only the service address is returned.
     conf = new HdfsConfiguration();
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR);
-    conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, NN1_ADDR);
     conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, NN1_SRVC_ADDR);
     
-    uris = DFSUtil.getNameServiceUris(conf, DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, 
-        DFS_NAMENODE_RPC_ADDRESS_KEY);
-    
-    assertEquals(1, uris.size());
-    assertTrue(uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR)));
+    uris = DFSUtil.getNameServiceUris(conf,
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
+
+    assertEquals("Incorrect number of URIs returned", 1, uris.size());
+    assertTrue("Missing URI for service address",
+        uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR)));
   }
 
   @Test (timeout=15000)


[39/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 0000000,7101753..d9f409c
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,1917 +1,1889 @@@
+ /**
+  * 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.FileSystem;
+ 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.core.SpanId;
+ import org.apache.htrace.core.TraceScope;
+ import org.apache.htrace.core.Tracer;
+ 
+ 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()).
+         setTracer(dfsClient.getTracer()).
+         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.newPathTraceScope("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.newPathTraceScope("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);
++      reportLostBlock(block, 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);
+   }
+ 
++  /**
++   * Warn the user of a lost block
++   */
++  protected void reportLostBlock(LocatedBlock lostBlock,
++      Collection<DatanodeInfo> ignoredNodes) {
++    DatanodeInfo[] nodes = lostBlock.getLocations();
++    DFSClient.LOG.warn("No live nodes contain block " + lostBlock.getBlock() +
++        " after checking nodes = " + Arrays.toString(nodes) +
++        ", ignoredNodes = " + ignoredNodes);
++  }
++
+   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 SpanId parentSpanId = Tracer.getCurrentSpanId();
+     return new Callable<ByteBuffer>() {
+       @Override
+       public ByteBuffer call() throws Exception {
+         byte[] buf = bb.array();
+         int offset = bb.position();
+         TraceScope scope = dfsClient.getTracer().
+             newScope("hedgedRead" + hedgedReadId, parentSpanId);
+         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 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 offset            the offset in buf
+    * @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,
++  void actualGetFromOneDataNode(final DNAddrPair datanode, LocatedBlock block,
++      final long startInBlk, final long endInBlk, byte[] buf, int offset,
+       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);
 -          }
++        int nread = reader.readAll(buf, offset, len);
++        updateReadStatistics(readStatistics, nread, reader);
++        if (nread != len) {
++          throw new IOException("truncated return from reader.read(): " +
++              "excpected " + len + ", 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, " 
++          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.
+         newPathTraceScope("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()) {
++        if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) {
+           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();
+   }
+ }


[41/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

Change-Id: Ic7946c4ea35bed587fe879ce58b959b25ecc0823


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

Branch: refs/heads/HDFS-7285
Commit: 8fd55202468b28422b0df888641c9b08906fe4a7
Parents: 6419900 151fca5
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Tue Sep 29 01:39:16 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 29 01:39:16 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       |   29 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |   33 +-
 hadoop-common-project/hadoop-common/pom.xml     |   45 +-
 .../hadoop/fs/CommonConfigurationKeys.java      |    3 +
 .../fs/CommonConfigurationKeysPublic.java       |    6 +
 .../org/apache/hadoop/fs/FSOutputSummer.java    |   12 +-
 .../java/org/apache/hadoop/fs/FileContext.java  |    7 +
 .../java/org/apache/hadoop/fs/FileSystem.java   |   27 +-
 .../org/apache/hadoop/fs/FilterFileSystem.java  |    1 -
 .../main/java/org/apache/hadoop/fs/FsShell.java |   26 +-
 .../java/org/apache/hadoop/fs/FsTracer.java     |   64 +
 .../main/java/org/apache/hadoop/fs/Globber.java |   15 +-
 .../org/apache/hadoop/fs/HarFileSystem.java     |    1 -
 .../org/apache/hadoop/fs/LocalDirAllocator.java |    4 +-
 .../apache/hadoop/fs/RawLocalFileSystem.java    |    1 -
 .../hadoop/fs/shell/CommandWithDestination.java |    8 +-
 .../hadoop/fs/viewfs/ChRootedFileSystem.java    |    1 -
 .../main/java/org/apache/hadoop/ipc/Client.java |   13 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |   17 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |   51 +-
 .../apache/hadoop/ipc/WritableRpcEngine.java    |   13 +-
 .../main/java/org/apache/hadoop/net/DNS.java    |  129 +-
 .../apache/hadoop/security/SecurityUtil.java    |   37 +-
 .../apache/hadoop/tracing/SpanReceiverHost.java |  208 --
 .../org/apache/hadoop/tracing/TraceUtils.java   |   24 +-
 .../tracing/TracerConfigurationManager.java     |  100 +
 .../java/org/apache/hadoop/util/ProtoUtil.java  |   13 +-
 .../src/main/proto/RpcHeader.proto              |    5 +-
 .../src/main/resources/core-default.xml         |   40 +
 .../hadoop-common/src/site/markdown/Metrics.md  |    1 -
 .../hadoop-common/src/site/markdown/Tracing.md  |   80 +-
 .../apache/hadoop/conf/TestConfiguration.java   |    2 +-
 .../java/org/apache/hadoop/fs/TestFsShell.java  |   44 +-
 .../org/apache/hadoop/fs/TestHarFileSystem.java |    1 -
 .../apache/hadoop/fs/TestLocalDirAllocator.java |   26 +-
 .../apache/hadoop/fs/TestLocalFileSystem.java   |    1 +
 .../AbstractContractRootDirectoryTest.java      |   14 +
 .../apache/hadoop/ha/ClientBaseWithFixes.java   |   40 +-
 .../http/TestAuthenticationSessionCookie.java   |    5 +-
 .../apache/hadoop/http/TestHttpCookieFlag.java  |    7 +-
 .../apache/hadoop/http/TestSSLHttpServer.java   |    4 +-
 .../java/org/apache/hadoop/ipc/TestRPC.java     |    4 +-
 .../org/apache/hadoop/net/ServerSocketUtil.java |    7 +-
 .../java/org/apache/hadoop/net/TestDNS.java     |  110 +-
 .../hadoop/security/TestSecurityUtil.java       |    2 +-
 .../hadoop/security/ssl/KeyStoreTestUtil.java   |   60 +-
 .../apache/hadoop/tracing/SetSpanReceiver.java  |   13 +-
 .../apache/hadoop/tracing/TestTraceUtils.java   |    2 +-
 .../dev-support/findbugsExcludeFile.xml         |   24 +
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 +
 .../hadoop/hdfs/BlockMissingException.java      |   65 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  906 +++++
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |   21 +-
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |   30 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3240 +++++++++++++++++
 .../hadoop/hdfs/DFSClientFaultInjector.java     |   60 +
 .../hadoop/hdfs/DFSHedgedReadMetrics.java       |   58 +
 .../hadoop/hdfs/DFSInotifyEventInputStream.java |  234 ++
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 1889 ++++++++++
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  982 ++++++
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  364 ++
 .../hadoop/hdfs/DFSStripedInputStream.java      |  972 ++++++
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  953 +++++
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   32 +
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 1944 +++++++++++
 .../hadoop/hdfs/DistributedFileSystem.java      | 2335 +++++++++++++
 .../apache/hadoop/hdfs/ExternalBlockReader.java |   20 +-
 .../hadoop/hdfs/HdfsConfigurationLoader.java    |   44 +
 .../hadoop/hdfs/NameNodeProxiesClient.java      |  366 ++
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   20 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |   23 +-
 .../apache/hadoop/hdfs/RemotePeerFactory.java   |   43 +
 .../org/apache/hadoop/hdfs/ReplicaAccessor.java |   10 +-
 .../hadoop/hdfs/ReplicaAccessorBuilder.java     |    3 +
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  187 +
 .../hdfs/UnknownCipherSuiteException.java       |   35 +
 .../UnknownCryptoProtocolVersionException.java  |   38 +
 .../org/apache/hadoop/hdfs/XAttrHelper.java     |  174 +
 .../hdfs/client/HdfsClientConfigKeys.java       |   11 +
 .../hadoop/hdfs/client/HdfsDataInputStream.java |  113 +
 .../hdfs/client/HdfsDataOutputStream.java       |  112 +
 .../client/impl/CorruptFileBlockIterator.java   |  105 +
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  524 +++
 .../hdfs/inotify/MissingEventsException.java    |   54 +
 .../hadoop/hdfs/protocol/AclException.java      |   39 +
 .../hdfs/protocol/CacheDirectiveIterator.java   |  129 +
 .../hadoop/hdfs/protocol/CachePoolIterator.java |   62 +
 .../hdfs/protocol/EncryptionZoneIterator.java   |   62 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   23 +
 .../QuotaByStorageTypeExceededException.java    |   56 +
 .../hdfs/protocol/UnresolvedPathException.java  |   87 +
 .../datatransfer/DataTransferProtoUtil.java     |   43 +-
 .../datatransfer/ReplaceDatanodeOnFailure.java  |  200 ++
 .../hdfs/protocol/datatransfer/Sender.java      |   22 +-
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 +
 .../ClientNamenodeProtocolTranslatorPB.java     | 1593 +++++++++
 .../datanode/ReplicaNotFoundException.java      |   53 +
 .../namenode/RetryStartFileException.java       |   36 +
 .../ha/AbstractNNFailoverProxyProvider.java     |   55 +
 .../ha/WrappedFailoverProxyProvider.java        |   71 +
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  952 +++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    1 -
 .../src/main/proto/hdfs.proto                   |  178 +-
 .../hadoop/hdfs/nfs/nfs3/OffsetRange.java       |    4 +
 .../hadoop/hdfs/nfs/nfs3/OpenFileCtx.java       |  141 +-
 .../apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java   |   82 +-
 .../apache/hadoop/hdfs/nfs/nfs3/TestWrites.java |   92 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   80 +-
 .../dev-support/findbugsExcludeFile.xml         |   19 -
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |    3 +-
 .../hadoop-hdfs/src/contrib/bkjournal/pom.xml   |    1 +
 .../bkjournal/src/main/proto/bkjournal.proto    |    1 +
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 -
 .../hadoop/hdfs/BlockMissingException.java      |   65 -
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  891 -----
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3244 ------------------
 .../hadoop/hdfs/DFSClientFaultInjector.java     |   57 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   39 +-
 .../hadoop/hdfs/DFSHedgedReadMetrics.java       |   58 -
 .../hadoop/hdfs/DFSInotifyEventInputStream.java |  239 --
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 1887 ----------
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  976 ------
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  359 --
 .../hadoop/hdfs/DFSStripedInputStream.java      |  973 ------
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  954 -----
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   31 -
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 1900 ----------
 .../hadoop/hdfs/DistributedFileSystem.java      | 2338 -------------
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |    9 +-
 .../apache/hadoop/hdfs/HdfsConfiguration.java   |   11 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |  275 +-
 .../apache/hadoop/hdfs/RemotePeerFactory.java   |   43 -
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  187 -
 .../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 -
 .../client/impl/CorruptFileBlockIterator.java   |  105 -
 .../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 -
 .../hdfs/protocol/datatransfer/Receiver.java    |   35 +-
 .../datatransfer/ReplaceDatanodeOnFailure.java  |  200 --
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 -
 .../ClientNamenodeProtocolTranslatorPB.java     | 1593 ---------
 .../DatanodeProtocolClientSideTranslatorPB.java |    2 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |    4 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |    4 +-
 .../NamenodeProtocolTranslatorPB.java           |    5 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   33 +-
 .../hdfs/qjournal/server/JournalNode.java       |   13 +-
 .../qjournal/server/JournalNodeRpcServer.java   |    1 +
 .../hadoop/hdfs/server/balancer/Dispatcher.java |    3 +-
 .../server/blockmanagement/DatanodeManager.java |   45 +-
 .../blockmanagement/HeartbeatManager.java       |   30 +-
 .../hdfs/server/common/HdfsServerConstants.java |   27 +-
 .../hdfs/server/datanode/BlockReceiver.java     |    4 +-
 .../hdfs/server/datanode/BlockSender.java       |   13 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   73 +-
 .../hdfs/server/datanode/DataXceiver.java       |    8 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |  337 +-
 .../datanode/ReplicaNotFoundException.java      |   53 -
 .../erasurecode/ErasureCodingWorker.java        |    6 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |    4 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |    8 +-
 .../impl/RamDiskAsyncLazyPersistService.java    |    4 +-
 .../hadoop/hdfs/server/namenode/BackupNode.java |    3 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |    2 +-
 .../server/namenode/FSDirEncryptionZoneOp.java  |  303 ++
 .../server/namenode/FSDirStatAndListingOp.java  |   27 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |    6 +-
 .../hdfs/server/namenode/FSDirectory.java       |  147 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  141 +-
 .../hdfs/server/namenode/LeaseManager.java      |    5 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |   25 +-
 .../hdfs/server/namenode/NameNodeMXBean.java    |    7 -
 .../hdfs/server/namenode/NameNodeRpcServer.java |   10 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |    5 +
 .../namenode/RetryStartFileException.java       |   36 -
 .../hdfs/server/namenode/TransferFsImage.java   |    4 +-
 .../ha/AbstractNNFailoverProxyProvider.java     |   51 -
 .../ha/ConfiguredFailoverProxyProvider.java     |    2 +-
 .../ha/WrappedFailoverProxyProvider.java        |   80 -
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |    2 +-
 .../hadoop/hdfs/tools/NNHAServiceTarget.java    |   16 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  946 -----
 .../main/native/fuse-dfs/test/TestFuseDFS.java  |    4 +-
 .../src/main/proto/DatanodeProtocol.proto       |    1 +
 .../hadoop-hdfs/src/main/proto/HdfsServer.proto |  198 ++
 .../src/main/proto/InterDatanodeProtocol.proto  |    1 +
 .../src/main/proto/JournalProtocol.proto        |    1 +
 .../src/main/proto/NamenodeProtocol.proto       |    1 +
 .../src/main/proto/QJournalProtocol.proto       |    1 +
 .../src/main/resources/hdfs-default.xml         |   58 +-
 .../hadoop-hdfs/src/site/markdown/Federation.md |    2 +-
 .../org/apache/hadoop/hdfs/TestFiPipelines.java |   17 +-
 .../datanode/TestFiDataTransferProtocol.java    |    4 +-
 .../datanode/TestFiDataTransferProtocol2.java   |    8 +-
 .../apache/hadoop/fs/TestFcHdfsSetUMask.java    |    4 +-
 .../TestSWebHdfsFileContextMainOperations.java  |    5 +
 .../TestWebHdfsFileContextMainOperations.java   |    5 +
 .../apache/hadoop/hdfs/BenchmarkThroughput.java |    3 +-
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |    2 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |    2 +-
 .../hdfs/MiniDFSClusterWithNodeGroup.java       |    2 +-
 .../hadoop/hdfs/TestBlockReaderLocal.java       |    2 +
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |    1 -
 .../TestClientProtocolForPipelineRecovery.java  |    6 +-
 .../apache/hadoop/hdfs/TestCrcCorruption.java   |    2 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |    3 +-
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |    5 +-
 .../org/apache/hadoop/hdfs/TestDFSPacket.java   |   25 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |   13 +
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |    8 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |   43 +
 .../hadoop/hdfs/TestDistributedFileSystem.java  |    3 +-
 .../hadoop/hdfs/TestExternalBlockReader.java    |   43 +-
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |    4 +-
 .../hadoop/hdfs/TestFileConcurrentReader.java   |    9 +-
 .../apache/hadoop/hdfs/TestFileCreation.java    |   12 +-
 .../hadoop/hdfs/TestFileCreationClient.java     |   11 +-
 .../org/apache/hadoop/hdfs/TestFileStatus.java  |    7 +-
 .../java/org/apache/hadoop/hdfs/TestHFlush.java |    6 +-
 .../org/apache/hadoop/hdfs/TestLargeBlock.java  |   10 +-
 .../java/org/apache/hadoop/hdfs/TestLease.java  |    8 +-
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  |   13 +-
 .../apache/hadoop/hdfs/TestListFilesInDFS.java  |    4 +-
 .../hadoop/hdfs/TestListFilesInFileContext.java |    4 +-
 .../org/apache/hadoop/hdfs/TestLocalDFS.java    |    7 +-
 .../apache/hadoop/hdfs/TestPersistBlocks.java   |    5 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |   10 +-
 .../hadoop/hdfs/TestReadWhileWriting.java       |    7 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   24 +-
 .../hdfs/qjournal/client/TestQJMWithFaults.java |    3 +-
 .../client/TestQuorumJournalManager.java        |    3 +-
 .../client/TestQuorumJournalManagerUnit.java    |    3 +-
 .../TestClientProtocolWithDelegationToken.java  |   12 +-
 .../hdfs/security/TestDelegationToken.java      |    4 +-
 .../security/token/block/TestBlockToken.java    |   12 +-
 .../hdfs/server/balancer/TestBalancer.java      |    3 +-
 .../TestBalancerWithMultipleNameNodes.java      |    4 +-
 .../blockmanagement/TestBlockTokenWithDFS.java  |    4 +-
 .../TestBlocksWithNotEnoughRacks.java           |    6 +-
 .../blockmanagement/TestHeartbeatHandling.java  |   27 +
 .../TestPendingInvalidateBlock.java             |    9 +-
 .../blockmanagement/TestReplicationPolicy.java  |    4 +-
 .../server/datanode/TestBPOfferService.java     |    3 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |    5 +-
 .../TestDataNodeVolumeFailureReporting.java     |    5 +-
 .../TestDatanodeProtocolRetryPolicy.java        |    3 +-
 .../server/datanode/TestDirectoryScanner.java   |  234 +-
 .../hdfs/server/datanode/TestTransferRbw.java   |    4 +-
 .../fsdataset/impl/TestSpaceReservation.java    |    5 +-
 .../fsdataset/impl/TestWriteToReplica.java      |    4 +-
 .../hdfs/server/mover/TestStorageMover.java     |   13 +-
 .../server/namenode/TestAuditLogAtDebug.java    |    4 +-
 .../hdfs/server/namenode/TestBackupNode.java    |    5 +-
 .../server/namenode/TestCacheDirectives.java    |    4 +-
 .../hdfs/server/namenode/TestCheckpoint.java    |    3 +-
 .../hdfs/server/namenode/TestEditLog.java       |    3 +-
 .../hdfs/server/namenode/TestEditLogRace.java   |    4 +-
 .../server/namenode/TestFSEditLogLoader.java    |    6 +-
 .../namenode/TestFSImageWithSnapshot.java       |    4 +-
 .../namenode/TestFavoredNodesEndToEnd.java      |    5 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |    5 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |    8 +-
 .../server/namenode/TestNameNodeMXBean.java     |    3 -
 .../namenode/TestNameNodeRetryCacheMetrics.java |    4 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |    3 +-
 .../ha/TestDNFencingWithReplication.java        |   11 +-
 .../server/namenode/ha/TestEditLogTailer.java   |    5 +-
 .../hdfs/server/namenode/ha/TestHAFsck.java     |    5 +-
 .../namenode/ha/TestHAStateTransitions.java     |    3 +-
 .../ha/TestLossyRetryInvocationHandler.java     |    8 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |    4 +-
 .../namenode/metrics/TestNameNodeMetrics.java   |    6 +-
 ...tINodeFileUnderConstructionWithSnapshot.java |   12 +-
 .../server/namenode/snapshot/TestSnapshot.java  |    3 +-
 .../hdfs/tools/TestDFSHAAdminMiniCluster.java   |    4 +-
 .../hadoop/hdfs/util/TestByteArrayManager.java  |   11 +-
 .../hdfs/web/TestFSMainOperationsWebHdfs.java   |    4 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |    4 +-
 .../web/TestWebHdfsWithMultipleNameNodes.java   |    3 +-
 .../hadoop/tools/TestHdfsConfigFields.java      |    6 +-
 .../apache/hadoop/tracing/TestTraceAdmin.java   |    9 +-
 .../org/apache/hadoop/tracing/TestTracing.java  |   86 +-
 .../TestTracingShortCircuitLocalRead.java       |   18 +-
 hadoop-mapreduce-project/CHANGES.txt            |   17 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |    4 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |    8 +
 .../v2/app/launcher/TestContainerLauncher.java  |   11 +
 .../app/launcher/TestContainerLauncherImpl.java |    9 +
 .../app/local/TestLocalContainerAllocator.java  |    6 +-
 .../v2/app/rm/TestRMContainerAllocator.java     |   19 +-
 .../java/org/apache/hadoop/mapred/Master.java   |   27 +-
 .../org/apache/hadoop/mapred/TestMaster.java    |   13 +
 hadoop-project/pom.xml                          |    6 +-
 .../dev-support/findbugs-exclude.xml            |   32 +
 hadoop-tools/hadoop-archive-logs/pom.xml        |   18 +
 .../apache/hadoop/tools/HadoopArchiveLogs.java  |  243 +-
 .../hadoop/tools/TestHadoopArchiveLogs.java     |  231 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |    3 +
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |    3 -
 .../fs/azure/NativeAzureFileSystemBaseTest.java |    1 -
 .../src/site/markdown/DistCp.md.vm              |    5 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |   14 +
 .../yarn/sls/scheduler/RMNodeWrapper.java       |   13 +
 .../sls/scheduler/ResourceSchedulerWrapper.java |   21 +-
 .../sls/scheduler/SLSCapacityScheduler.java     |   19 +-
 .../org/apache/hadoop/streaming/UtilTest.java   |    5 -
 hadoop-yarn-project/CHANGES.txt                 |   80 +-
 .../yarn/api/ContainerManagementProtocol.java   |   30 +-
 .../api/protocolrecords/AllocateRequest.java    |   57 +-
 .../api/protocolrecords/AllocateResponse.java   |   42 +-
 .../IncreaseContainersResourceRequest.java      |   75 +
 .../IncreaseContainersResourceResponse.java     |   93 +
 .../records/ContainerResourceChangeRequest.java |  117 +
 .../api/records/ContainerResourceDecrease.java  |   78 -
 .../api/records/ContainerResourceIncrease.java  |   84 -
 .../ContainerResourceIncreaseRequest.java       |   80 -
 .../yarn/api/records/ContainerStatus.java       |   13 +
 .../proto/containermanagement_protocol.proto    |    1 +
 .../src/main/proto/yarn_protos.proto            |   14 +-
 .../src/main/proto/yarn_service_protos.proto    |   16 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |    7 +-
 .../yarn/client/TestResourceTrackerOnHA.java    |    2 +-
 .../api/impl/TestAMRMClientOnRMRestart.java     |    8 +-
 ...ContainerManagementProtocolPBClientImpl.java |   20 +
 ...ontainerManagementProtocolPBServiceImpl.java |   22 +
 .../impl/pb/AllocateRequestPBImpl.java          |  119 +-
 .../impl/pb/AllocateResponsePBImpl.java         |  175 +-
 ...IncreaseContainersResourceRequestPBImpl.java |  170 +
 ...ncreaseContainersResourceResponsePBImpl.java |  241 ++
 .../ContainerResourceChangeRequestPBImpl.java   |  141 +
 .../pb/ContainerResourceDecreasePBImpl.java     |  136 -
 .../pb/ContainerResourceIncreasePBImpl.java     |  171 -
 .../ContainerResourceIncreaseRequestPBImpl.java |  141 -
 .../records/impl/pb/ContainerStatusPBImpl.java  |   31 +-
 .../resource/DefaultResourceCalculator.java     |    5 +
 .../resource/DominantResourceCalculator.java    |    6 +
 .../yarn/util/resource/ResourceCalculator.java  |    5 +
 .../hadoop/yarn/util/resource/Resources.java    |    5 +
 .../hadoop/yarn/TestContainerLaunchRPC.java     |    8 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |  162 +
 .../java/org/apache/hadoop/yarn/TestRPC.java    |    8 +
 .../hadoop/yarn/api/TestAllocateRequest.java    |   73 -
 .../hadoop/yarn/api/TestAllocateResponse.java   |  114 -
 .../yarn/api/TestContainerResourceDecrease.java |   66 -
 .../yarn/api/TestContainerResourceIncrease.java |   74 -
 .../TestContainerResourceIncreaseRequest.java   |   68 -
 .../hadoop/yarn/api/TestPBImplRecords.java      |   54 +-
 .../util/resource/TestResourceCalculator.java   |   30 +-
 ...pplicationHistoryManagerOnTimelineStore.java |   12 +
 .../ApplicationHistoryServer.java               |   13 +-
 ...pplicationHistoryManagerOnTimelineStore.java |   43 +-
 .../TestApplicationHistoryServer.java           |    4 +-
 .../protocolrecords/NodeHeartbeatResponse.java  |    5 +
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |   77 +-
 .../yarn/server/api/records/NodeStatus.java     |   15 +-
 .../api/records/impl/pb/NodeStatusPBImpl.java   |   75 +-
 .../metrics/ApplicationMetricsConstants.java    |    3 +
 .../hadoop/yarn/server/utils/BuilderUtils.java  |    4 +-
 .../main/proto/yarn_server_common_protos.proto  |    3 +-
 .../yarn_server_common_service_protos.proto     |    1 +
 .../hadoop/yarn/TestYarnServerApiClasses.java   |   39 +-
 .../CMgrDecreaseContainersResourceEvent.java    |   37 +
 .../nodemanager/ContainerManagerEventType.java  |    1 +
 .../hadoop/yarn/server/nodemanager/Context.java |    3 +
 .../nodemanager/LocalDirsHandlerService.java    |   33 +-
 .../yarn/server/nodemanager/NodeManager.java    |   10 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |   57 +-
 .../containermanager/ContainerManagerImpl.java  |  302 +-
 .../containermanager/container/Container.java   |    2 +
 .../container/ContainerImpl.java                |   24 +-
 .../logaggregation/LogAggregationService.java   |   11 +-
 .../ChangeMonitoringContainerResourceEvent.java |   37 +
 .../monitor/ContainersMonitorEventType.java     |    3 +-
 .../monitor/ContainersMonitorImpl.java          |  207 +-
 .../nodemanager/metrics/NodeManagerMetrics.java |   11 +
 .../recovery/NMLeveldbStateStoreService.java    |   22 +
 .../recovery/NMNullStateStoreService.java       |    6 +
 .../recovery/NMStateStoreService.java           |   15 +
 .../nodemanager/DummyContainerManager.java      |    6 +-
 .../TestContainerManagerWithLCE.java            |   33 +
 .../TestLocalDirsHandlerService.java            |   18 +
 .../nodemanager/TestNodeManagerResync.java      |  258 ++
 .../nodemanager/TestNodeStatusUpdater.java      |    2 +-
 .../amrmproxy/BaseAMRMProxyTest.java            |    5 +
 .../amrmproxy/MockResourceManagerFacade.java    |    6 +-
 .../BaseContainerManagerTest.java               |   43 +-
 .../containermanager/TestContainerManager.java  |  288 +-
 .../TestContainerManagerRecovery.java           |  233 +-
 .../TestLogAggregationService.java              |   19 +
 .../monitor/MockResourceCalculatorPlugin.java   |   69 +
 .../MockResourceCalculatorProcessTree.java      |   57 +
 .../TestContainersMonitorResourceChange.java    |  248 ++
 .../metrics/TestNodeManagerMetrics.java         |   18 +-
 .../recovery/NMMemoryStateStoreService.java     |   11 +-
 .../TestNMLeveldbStateStoreService.java         |   11 +
 .../nodemanager/webapp/MockContainer.java       |    4 +
 .../ApplicationMasterService.java               |   22 +-
 .../server/resourcemanager/ClientRMService.java |   30 +-
 .../server/resourcemanager/RMAuditLogger.java   |    2 +
 .../server/resourcemanager/RMServerUtils.java   |  164 +
 .../resourcemanager/ResourceTrackerService.java |    7 +-
 .../resourcemanager/amlauncher/AMLauncher.java  |   27 +-
 .../metrics/ApplicationUpdatedEvent.java        |   54 +
 .../metrics/SystemMetricsEventType.java         |    1 +
 .../metrics/SystemMetricsPublisher.java         |   29 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |    5 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |    4 +-
 .../rmcontainer/RMContainer.java                |    4 +
 .../RMContainerChangeResourceEvent.java         |   44 +
 .../rmcontainer/RMContainerEventType.java       |   13 +-
 .../rmcontainer/RMContainerImpl.java            |  121 +-
 .../RMContainerUpdatesAcquiredEvent.java        |   35 +
 .../server/resourcemanager/rmnode/RMNode.java   |    9 +
 .../rmnode/RMNodeDecreaseContainerEvent.java    |   39 +
 .../resourcemanager/rmnode/RMNodeEventType.java |    1 +
 .../resourcemanager/rmnode/RMNodeImpl.java      |   93 +
 .../rmnode/RMNodeStatusEvent.java               |   33 +-
 .../scheduler/AbstractYarnScheduler.java        |  150 +-
 .../resourcemanager/scheduler/Allocation.java   |   22 +-
 .../scheduler/AppSchedulingInfo.java            |  249 +-
 .../resourcemanager/scheduler/QueueMetrics.java |   16 +-
 .../scheduler/SchedContainerChangeRequest.java  |  118 +
 .../scheduler/SchedulerApplication.java         |    2 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  250 +-
 .../scheduler/SchedulerNode.java                |   31 +
 .../scheduler/SchedulerUtils.java               |   11 +-
 .../scheduler/YarnScheduler.java                |   14 +-
 .../scheduler/capacity/AbstractCSQueue.java     |   23 +-
 .../scheduler/capacity/CSAssignment.java        |    9 +
 .../scheduler/capacity/CSQueue.java             |   16 +
 .../scheduler/capacity/CapacityScheduler.java   |   87 +-
 .../scheduler/capacity/LeafQueue.java           |  127 +-
 .../scheduler/capacity/ParentQueue.java         |  115 +-
 .../allocator/AbstractContainerAllocator.java   |  131 +
 .../capacity/allocator/ContainerAllocator.java  |  149 +-
 .../allocator/IncreaseContainerAllocator.java   |  365 ++
 .../allocator/RegularContainerAllocator.java    |   30 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |   68 +-
 .../scheduler/fair/FSLeafQueue.java             |    5 +-
 .../scheduler/fair/FSParentQueue.java           |    3 +-
 .../scheduler/fair/FairScheduler.java           |   35 +-
 .../scheduler/fair/QueueManager.java            |   24 +-
 .../scheduler/fifo/FifoScheduler.java           |   25 +-
 .../server/resourcemanager/Application.java     |    2 +-
 .../yarn/server/resourcemanager/MockAM.java     |    9 +
 .../yarn/server/resourcemanager/MockNM.java     |    2 +-
 .../yarn/server/resourcemanager/MockNodes.java  |   13 +
 .../yarn/server/resourcemanager/MockRM.java     |   25 +-
 .../server/resourcemanager/NodeManager.java     |   13 +-
 .../resourcemanager/TestAMAuthorization.java    |    8 +
 .../resourcemanager/TestApplicationCleanup.java |    6 +-
 .../TestApplicationMasterLauncher.java          |   75 +
 .../TestApplicationMasterService.java           |  144 +-
 .../resourcemanager/TestClientRMService.java    |   36 +-
 .../applicationsmanager/TestAMRestart.java      |   15 +-
 .../TestRMAppLogAggregationStatus.java          |   10 +-
 .../metrics/TestSystemMetricsPublisher.java     |   50 +-
 .../attempt/TestRMAppAttemptTransitions.java    |   53 +-
 .../rmcontainer/TestRMContainerImpl.java        |  117 +-
 .../capacity/TestCapacityScheduler.java         |  130 +-
 .../scheduler/capacity/TestChildQueueOrder.java |    4 +-
 .../capacity/TestContainerAllocation.java       |   50 +-
 .../capacity/TestContainerResizing.java         |  963 ++++++
 .../scheduler/capacity/TestLeafQueue.java       |    4 +-
 .../scheduler/capacity/TestParentQueue.java     |    4 +-
 .../scheduler/capacity/TestReservations.java    |    9 +-
 .../scheduler/fair/FairSchedulerTestBase.java   |    6 +-
 .../fair/TestContinuousScheduling.java          |    2 +-
 .../scheduler/fair/TestFSParentQueue.java       |   79 +
 .../scheduler/fair/TestFairScheduler.java       |   30 +-
 .../scheduler/fifo/TestFifoScheduler.java       |   32 +-
 .../security/TestAMRMTokens.java                |    3 +-
 .../yarn/server/webproxy/AppReportFetcher.java  |   49 +-
 .../server/webproxy/WebAppProxyServlet.java     |   46 +-
 .../server/webproxy/TestWebAppProxyServlet.java |  113 +-
 485 files changed, 31104 insertions(+), 23102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
----------------------------------------------------------------------
diff --cc hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index a8a7494,648043e..616ff3a
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@@ -196,12 -198,8 +198,12 @@@ abstract public class FSOutputSummer ex
      return sum.getChecksumSize();
    }
  
 +  protected DataChecksum getDataChecksum() {
 +    return sum;
 +  }
 +
    protected TraceScope createWriteTraceScope() {
-     return NullScope.INSTANCE;
+     return null;
    }
  
    /** Generate checksums for the given data chunks and output chunks & checksums

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------


[34/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index 0000000,0000000..a313ecb
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@@ -1,0 -1,0 +1,187 @@@
++/**
++ * 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.concurrent.atomic.AtomicReference;
++
++import org.apache.hadoop.fs.StorageType;
++import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
++import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
++import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
++import org.apache.hadoop.hdfs.protocol.LocatedBlock;
++import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
++import org.apache.hadoop.hdfs.util.ByteArrayManager;
++import org.apache.hadoop.io.erasurecode.ECSchema;
++import org.apache.hadoop.util.DataChecksum;
++import org.apache.hadoop.util.Progressable;
++
++import com.google.common.annotations.VisibleForTesting;
++
++/**
++ * This class extends {@link DataStreamer} to support writing striped blocks
++ * to datanodes.
++ * A {@link DFSStripedOutputStream} has multiple {@link StripedDataStreamer}s.
++ * Whenever the streamers need to talk the namenode, only the fastest streamer
++ * sends an rpc call to the namenode and then populates the result for the
++ * other streamers.
++ */
++public class StripedDataStreamer extends DataStreamer {
++  private final Coordinator coordinator;
++  private final int index;
++
++  StripedDataStreamer(HdfsFileStatus stat,
++                      DFSClient dfsClient, String src,
++                      Progressable progress, DataChecksum checksum,
++                      AtomicReference<CachingStrategy> cachingStrategy,
++                      ByteArrayManager byteArrayManage, String[] favoredNodes,
++                      short index, Coordinator coordinator) {
++    super(stat, null, dfsClient, src, progress, checksum, cachingStrategy,
++        byteArrayManage, favoredNodes);
++    this.index = index;
++    this.coordinator = coordinator;
++  }
++
++  int getIndex() {
++    return index;
++  }
++
++  boolean isHealthy() {
++    return !streamerClosed() && !getErrorState().hasInternalError();
++  }
++
++  @Override
++  protected void endBlock() {
++    coordinator.offerEndBlock(index, block);
++    super.endBlock();
++  }
++
++  /**
++   * The upper level DFSStripedOutputStream will allocate the new block group.
++   * All the striped data streamer only needs to fetch from the queue, which
++   * should be already be ready.
++   */
++  private LocatedBlock getFollowingBlock() throws IOException {
++    if (!this.isHealthy()) {
++      // No internal block for this streamer, maybe no enough healthy DN.
++      // Throw the exception which has been set by the StripedOutputStream.
++      this.getLastException().check(false);
++    }
++    return coordinator.getFollowingBlocks().poll(index);
++  }
++
++  @Override
++  protected LocatedBlock nextBlockOutputStream() throws IOException {
++    boolean success;
++    LocatedBlock lb = getFollowingBlock();
++    block = lb.getBlock();
++    block.setNumBytes(0);
++    bytesSent = 0;
++    accessToken = lb.getBlockToken();
++
++    DatanodeInfo[] nodes = lb.getLocations();
++    StorageType[] storageTypes = lb.getStorageTypes();
++
++    // Connect to the DataNode. If fail the internal error state will be set.
++    success = createBlockOutputStream(nodes, storageTypes, 0L, false);
++
++    if (!success) {
++      block = null;
++      final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()];
++      LOG.info("Excluding datanode " + badNode);
++      excludedNodes.put(badNode, badNode);
++      throw new IOException("Unable to create new block.");
++    }
++    return lb;
++  }
++
++  @VisibleForTesting
++  LocatedBlock peekFollowingBlock() {
++    return coordinator.getFollowingBlocks().peek(index);
++  }
++
++  @Override
++  protected void setupPipelineInternal(DatanodeInfo[] nodes,
++      StorageType[] nodeStorageTypes) throws IOException {
++    boolean success = false;
++    while (!success && !streamerClosed() && dfsClient.clientRunning) {
++      if (!handleRestartingDatanode()) {
++        return;
++      }
++      if (!handleBadDatanode()) {
++        // for striped streamer if it is datanode error then close the stream
++        // and return. no need to replace datanode
++        return;
++      }
++
++      // get a new generation stamp and an access token
++      final LocatedBlock lb = coordinator.getNewBlocks().take(index);
++      long newGS = lb.getBlock().getGenerationStamp();
++      setAccessToken(lb.getBlockToken());
++
++      // set up the pipeline again with the remaining nodes. when a striped
++      // data streamer comes here, it must be in external error state.
++      assert getErrorState().hasExternalError();
++      success = createBlockOutputStream(nodes, nodeStorageTypes, newGS, true);
++
++      failPacket4Testing();
++      getErrorState().checkRestartingNodeDeadline(nodes);
++
++      // notify coordinator the result of createBlockOutputStream
++      synchronized (coordinator) {
++        if (!streamerClosed()) {
++          coordinator.updateStreamer(this, success);
++          coordinator.notify();
++        } else {
++          success = false;
++        }
++      }
++
++      if (success) {
++        // wait for results of other streamers
++        success = coordinator.takeStreamerUpdateResult(index);
++        if (success) {
++          // if all succeeded, update its block using the new GS
++          block = newBlock(block, newGS);
++        } else {
++          // otherwise close the block stream and restart the recovery process
++          closeStream();
++        }
++      } else {
++        // if fail, close the stream. The internal error state and last
++        // exception have already been set in createBlockOutputStream
++        // TODO: wait for restarting DataNodes during RollingUpgrade
++        closeStream();
++        setStreamerAsClosed();
++      }
++    } // while
++  }
++
++  void setExternalError() {
++    getErrorState().setExternalError();
++    synchronized (dataQueue) {
++      dataQueue.notifyAll();
++    }
++  }
++
++  @Override
++  public String toString() {
++    return "#" + index + ": " + (!isHealthy() ? "failed, ": "") + super.toString();
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 0000000,f4ce46d..7b02691
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@@ -1,0 -1,1531 +1,1593 @@@
+ /**
+  * 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.protocolPB;
+ 
+ import java.io.Closeable;
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.util.Arrays;
+ import java.util.EnumSet;
+ import java.util.List;
+ 
+ import com.google.common.collect.Lists;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+ 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.FsServerDefaults;
+ import org.apache.hadoop.fs.Options.Rename;
+ import org.apache.hadoop.fs.ParentNotDirectoryException;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.UnresolvedLinkException;
+ 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.inotify.EventBatchList;
+ import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+ 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.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ 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.ExtendedBlock;
+ 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.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
 -import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
++import org.apache.hadoop.hdfs.protocol.proto.*;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
+ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.io.Text;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+ import org.apache.hadoop.ipc.ProtobufHelper;
+ import org.apache.hadoop.ipc.ProtocolMetaInterface;
+ import org.apache.hadoop.ipc.ProtocolTranslator;
+ import org.apache.hadoop.ipc.RPC;
+ import org.apache.hadoop.ipc.RpcClientUtil;
+ import org.apache.hadoop.security.AccessControlException;
+ import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
+ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
+ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
+ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
+ import org.apache.hadoop.security.token.Token;
+ 
+ import com.google.protobuf.ByteString;
+ import com.google.protobuf.ServiceException;
+ 
+ import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+ import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
+     .EncryptionZoneProto;
+ 
+ /**
+  * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
+  * while translating from the parameter types used in ClientProtocol to the
+  * new PB types.
+  */
+ @InterfaceAudience.Private
+ @InterfaceStability.Stable
+ public class ClientNamenodeProtocolTranslatorPB implements
+     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
+   final private ClientNamenodeProtocolPB rpcProxy;
+ 
+   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = 
+   GetServerDefaultsRequestProto.newBuilder().build();
+ 
+   private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
+   GetFsStatusRequestProto.newBuilder().build();
+ 
+   private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
+   SaveNamespaceRequestProto.newBuilder().build();
+ 
+   private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = 
+   RollEditsRequestProto.getDefaultInstance();
+ 
+   private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
+   RefreshNodesRequestProto.newBuilder().build();
+ 
+   private final static FinalizeUpgradeRequestProto
+   VOID_FINALIZE_UPGRADE_REQUEST =
+       FinalizeUpgradeRequestProto.newBuilder().build();
+ 
+   private final static GetDataEncryptionKeyRequestProto
+   VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
+       GetDataEncryptionKeyRequestProto.newBuilder().build();
+ 
+   private final static GetStoragePoliciesRequestProto
+   VOID_GET_STORAGE_POLICIES_REQUEST =
+       GetStoragePoliciesRequestProto.newBuilder().build();
+ 
++  private final static GetErasureCodingPoliciesRequestProto
++  VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto
++      .newBuilder().build();
++
+   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
+     rpcProxy = proxy;
+   }
+   
+   @Override
+   public void close() {
+     RPC.stopProxy(rpcProxy);
+   }
+ 
+   @Override
+   public LocatedBlocks getBlockLocations(String src, long offset, long length)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
+         .newBuilder()
+         .setSrc(src)
+         .setOffset(offset)
+         .setLength(length)
+         .build();
+     try {
+       GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
+           req);
+       return resp.hasLocations() ?
+         PBHelperClient.convert(resp.getLocations()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public FsServerDefaults getServerDefaults() throws IOException {
+     GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST;
+     try {
+       return PBHelperClient
+           .convert(rpcProxy.getServerDefaults(null, req).getServerDefaults());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public HdfsFileStatus create(String src, FsPermission masked,
+       String clientName, EnumSetWritable<CreateFlag> flag,
+       boolean createParent, short replication, long blockSize, 
+       CryptoProtocolVersion[] supportedVersions)
+       throws AccessControlException, AlreadyBeingCreatedException,
+       DSQuotaExceededException, FileAlreadyExistsException,
+       FileNotFoundException, NSQuotaExceededException,
+       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
+         .setSrc(src)
+         .setMasked(PBHelperClient.convert(masked))
+         .setClientName(clientName)
+         .setCreateFlag(PBHelperClient.convertCreateFlag(flag))
+         .setCreateParent(createParent)
+         .setReplication(replication)
+         .setBlockSize(blockSize);
+     builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
+     CreateRequestProto req = builder.build();
+     try {
+       CreateResponseProto res = rpcProxy.create(null, req);
+       return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+ 
+   }
+ 
+   @Override
+   public boolean truncate(String src, long newLength, String clientName)
+       throws IOException, UnresolvedLinkException {
+     TruncateRequestProto req = TruncateRequestProto.newBuilder()
+         .setSrc(src)
+         .setNewLength(newLength)
+         .setClientName(clientName)
+         .build();
+     try {
+       return rpcProxy.truncate(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public LastBlockWithStatus append(String src, String clientName,
+       EnumSetWritable<CreateFlag> flag) throws AccessControlException,
+       DSQuotaExceededException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
+         .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
+         .build();
+     try {
+       AppendResponseProto res = rpcProxy.append(null, req);
+       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
 -          .convert(res.getBlock()) : null;
++          .convertLocatedBlockProto(res.getBlock()) : null;
+       HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
+           : null;
+       return new LastBlockWithStatus(lastBlock, stat);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean setReplication(String src, short replication)
+       throws AccessControlException, DSQuotaExceededException,
+       FileNotFoundException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
+         .setSrc(src)
+         .setReplication(replication)
+         .build();
+     try {
+       return rpcProxy.setReplication(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setPermission(String src, FsPermission permission)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
+         .setSrc(src)
+         .setPermission(PBHelperClient.convert(permission))
+         .build();
+     try {
+       rpcProxy.setPermission(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setOwner(String src, String username, String groupname)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
+         .setSrc(src);
+     if (username != null)
+         req.setUsername(username);
+     if (groupname != null)
+         req.setGroupname(groupname);
+     try {
+       rpcProxy.setOwner(null, req.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void abandonBlock(ExtendedBlock b, long fileId, String src,
+       String holder) throws AccessControlException, FileNotFoundException,
+         UnresolvedLinkException, IOException {
+     AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
+         .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
+             .setFileId(fileId).build();
+     try {
+       rpcProxy.abandonBlock(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public LocatedBlock addBlock(String src, String clientName,
+       ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
+       String[] favoredNodes)
+       throws AccessControlException, FileNotFoundException,
+       NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
+         .setSrc(src).setClientName(clientName).setFileId(fileId);
+     if (previous != null) 
+       req.setPrevious(PBHelperClient.convert(previous));
+     if (excludeNodes != null)
+       req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
+     if (favoredNodes != null) {
+       req.addAllFavoredNodes(Arrays.asList(favoredNodes));
+     }
+     try {
 -      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
++          rpcProxy.addBlock(null, req.build()).getBlock());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public LocatedBlock getAdditionalDatanode(String src, long fileId,
+       ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
+       DatanodeInfo[] excludes,
+       int numAdditionalNodes, String clientName) throws AccessControlException,
+       FileNotFoundException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
+         .newBuilder()
+         .setSrc(src)
+         .setFileId(fileId)
+         .setBlk(PBHelperClient.convert(blk))
+         .addAllExistings(PBHelperClient.convert(existings))
+         .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
+         .addAllExcludes(PBHelperClient.convert(excludes))
+         .setNumAdditionalNodes(numAdditionalNodes)
+         .setClientName(clientName)
+         .build();
+     try {
 -      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
 -          .getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
++          rpcProxy.getAdditionalDatanode(null, req).getBlock());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean complete(String src, String clientName,
+                           ExtendedBlock last, long fileId)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
+         .setSrc(src)
+         .setClientName(clientName)
+         .setFileId(fileId);
+     if (last != null)
+       req.setLast(PBHelperClient.convert(last));
+     try {
+       return rpcProxy.complete(null, req.build()).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
 -        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
++        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks)))
+         .build();
+     try {
+       rpcProxy.reportBadBlocks(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean rename(String src, String dst) throws UnresolvedLinkException,
+       IOException {
+     RenameRequestProto req = RenameRequestProto.newBuilder()
+         .setSrc(src)
+         .setDst(dst).build();
+     try {
+       return rpcProxy.rename(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+ 
+   @Override
+   public void rename2(String src, String dst, Rename... options)
+       throws AccessControlException, DSQuotaExceededException,
+       FileAlreadyExistsException, FileNotFoundException,
+       NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     boolean overwrite = false;
+     if (options != null) {
+       for (Rename option : options) {
+         if (option == Rename.OVERWRITE) {
+           overwrite = true;
+         }
+       }
+     }
+     Rename2RequestProto req = Rename2RequestProto.newBuilder().
+         setSrc(src).
+         setDst(dst).setOverwriteDest(overwrite).
+         build();
+     try {
+       rpcProxy.rename2(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+ 
+   }
+ 
+   @Override
+   public void concat(String trg, String[] srcs) throws IOException,
+       UnresolvedLinkException {
+     ConcatRequestProto req = ConcatRequestProto.newBuilder().
+         setTrg(trg).
+         addAllSrcs(Arrays.asList(srcs)).build();
+     try {
+       rpcProxy.concat(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+ 
+   @Override
+   public boolean delete(String src, boolean recursive)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
+     try {
+       return rpcProxy.delete(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+       throws AccessControlException, FileAlreadyExistsException,
+       FileNotFoundException, NSQuotaExceededException,
+       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
+         .setSrc(src)
+         .setMasked(PBHelperClient.convert(masked))
+         .setCreateParent(createParent).build();
+ 
+     try {
+       return rpcProxy.mkdirs(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public DirectoryListing getListing(String src, byte[] startAfter,
+       boolean needLocation) throws AccessControlException,
+       FileNotFoundException, UnresolvedLinkException, IOException {
+     GetListingRequestProto req = GetListingRequestProto.newBuilder()
+         .setSrc(src)
+         .setStartAfter(ByteString.copyFrom(startAfter))
+         .setNeedLocation(needLocation).build();
+     try {
+       GetListingResponseProto result = rpcProxy.getListing(null, req);
+       
+       if (result.hasDirList()) {
+         return PBHelperClient.convert(result.getDirList());
+       }
+       return null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void renewLease(String clientName) throws AccessControlException,
+       IOException {
+     RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
+         .setClientName(clientName).build();
+     try {
+       rpcProxy.renewLease(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean recoverLease(String src, String clientName)
+       throws IOException {
+     RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder()
+         .setSrc(src)
+         .setClientName(clientName).build();
+     try {
+       return rpcProxy.recoverLease(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }  
+   }
+ 
+   @Override
+   public long[] getStats() throws IOException {
+     try {
+       return PBHelperClient.convert(rpcProxy.getFsStats(null,
+           VOID_GET_FSSTATUS_REQUEST));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
+       throws IOException {
+     GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto
+         .newBuilder()
+         .setType(PBHelperClient.convert(type)).build();
+     try {
+       return PBHelperClient.convert(
+           rpcProxy.getDatanodeReport(null, req).getDiList());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
+       throws IOException {
+     final GetDatanodeStorageReportRequestProto req
+         = GetDatanodeStorageReportRequestProto.newBuilder()
+             .setType(PBHelperClient.convert(type)).build();
+     try {
+       return PBHelperClient.convertDatanodeStorageReports(
+           rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public long getPreferredBlockSize(String filename) throws IOException,
+       UnresolvedLinkException {
+     GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
+         .newBuilder()
+         .setFilename(filename)
+         .build();
+     try {
+       return rpcProxy.getPreferredBlockSize(null, req).getBsize();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
+     SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
+         .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
+     try {
+       return rpcProxy.setSafeMode(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+     try {
+       SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder()
+           .setTimeWindow(timeWindow).setTxGap(txGap).build();
+       return rpcProxy.saveNamespace(null, req).getSaved();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public long rollEdits() throws AccessControlException, IOException {
+     try {
+       RollEditsResponseProto resp = rpcProxy.rollEdits(null,
+           VOID_ROLLEDITS_REQUEST);
+       return resp.getNewSegmentTxId();
+     } catch (ServiceException se) {
+       throw ProtobufHelper.getRemoteException(se);
+     }
+   }
+ 
+   @Override
+   public boolean restoreFailedStorage(String arg) 
+       throws AccessControlException, IOException{
+     RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
+         .newBuilder()
+         .setArg(arg).build();
+     try {
+       return rpcProxy.restoreFailedStorage(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void refreshNodes() throws IOException {
+     try {
+       rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void finalizeUpgrade() throws IOException {
+     try {
+       rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+     final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
+         .setAction(PBHelperClient.convert(action)).build();
+     try {
+       final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
+       if (proto.hasRollingUpgradeInfo()) {
+         return PBHelperClient.convert(proto.getRollingUpgradeInfo());
+       }
+       return null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+       throws IOException {
+     ListCorruptFileBlocksRequestProto.Builder req = 
+         ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);   
+     if (cookie != null) 
+       req.setCookie(cookie);
+     try {
+       return PBHelperClient.convert(
+           rpcProxy.listCorruptFileBlocks(null, req.build()).getCorrupt());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void metaSave(String filename) throws IOException {
+     MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder()
+         .setFilename(filename).build();
+     try {
+       rpcProxy.metaSave(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+ 
+   }
+ 
+   @Override
+   public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+       FileNotFoundException, UnresolvedLinkException, IOException {
+     GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
+       return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public HdfsFileStatus getFileLinkInfo(String src)
+       throws AccessControlException, UnresolvedLinkException, IOException {
+     GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
+       return result.hasFs() ?  
+           PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public ContentSummary getContentSummary(String path)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     GetContentSummaryRequestProto req = GetContentSummaryRequestProto
+         .newBuilder()
+         .setPath(path)
+         .build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getContentSummary(null, req)
+           .getSummary());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+                        StorageType type)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     final SetQuotaRequestProto.Builder builder
+         = SetQuotaRequestProto.newBuilder()
+         .setPath(path)
+         .setNamespaceQuota(namespaceQuota)
+         .setStoragespaceQuota(storagespaceQuota);
+     if (type != null) {
+       builder.setStorageType(PBHelperClient.convertStorageType(type));
+     }
+     final SetQuotaRequestProto req = builder.build();
+     try {
+       rpcProxy.setQuota(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void fsync(String src, long fileId, String client,
+                     long lastBlockLength)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
+         .setClient(client).setLastBlockLength(lastBlockLength)
+             .setFileId(fileId).build();
+     try {
+       rpcProxy.fsync(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setTimes(String src, long mtime, long atime)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
+         .setSrc(src)
+         .setMtime(mtime)
+         .setAtime(atime)
+         .build();
+     try {
+       rpcProxy.setTimes(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void createSymlink(String target, String link, FsPermission dirPerm,
+       boolean createParent) throws AccessControlException,
+       FileAlreadyExistsException, FileNotFoundException,
+       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
+         .setTarget(target)
+         .setLink(link)
+         .setDirPerm(PBHelperClient.convert(dirPerm))
+         .setCreateParent(createParent)
+         .build();
+     try {
+       rpcProxy.createSymlink(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public String getLinkTarget(String path) throws AccessControlException,
+       FileNotFoundException, IOException {
+     GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
+         .setPath(path).build();
+     try {
+       GetLinkTargetResponseProto rsp = rpcProxy.getLinkTarget(null, req);
+       return rsp.hasTargetPath() ? rsp.getTargetPath() : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+       String clientName) throws IOException {
+     UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto
+         .newBuilder()
+         .setBlock(PBHelperClient.convert(block))
+         .setClientName(clientName)
+         .build();
+     try {
 -      return PBHelperClient.convert(
++      return PBHelperClient.convertLocatedBlockProto(
+           rpcProxy.updateBlockForPipeline(null, req).getBlock());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
+     UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
+         .setClientName(clientName)
+         .setOldBlock(PBHelperClient.convert(oldBlock))
+         .setNewBlock(PBHelperClient.convert(newBlock))
+         .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes)))
+         .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
+         .build();
+     try {
+       rpcProxy.updatePipeline(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+       throws IOException {
+     GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto
+         .newBuilder()
+         .setRenewer(renewer == null ? "" : renewer.toString())
+         .build();
+     try {
+       GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
+       return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
+           : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException {
+     RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
+         setToken(PBHelperClient.convert(token)).
+         build();
+     try {
+       return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException {
+     CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto
+         .newBuilder()
+         .setToken(PBHelperClient.convert(token))
+         .build();
+     try {
+       rpcProxy.cancelDelegationToken(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setBalancerBandwidth(long bandwidth) throws IOException {
+     SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
+         .setBandwidth(bandwidth)
+         .build();
+     try {
+       rpcProxy.setBalancerBandwidth(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean isMethodSupported(String methodName) throws IOException {
+     return RpcClientUtil.isMethodSupported(rpcProxy,
+         ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+         RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
+   }
+   
+   @Override
+   public DataEncryptionKey getDataEncryptionKey() throws IOException {
+     try {
+       GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
+           null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
+      return rsp.hasDataEncryptionKey() ? 
+           PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+ 
+   @Override
+   public boolean isFileClosed(String src) throws AccessControlException,
+       FileNotFoundException, UnresolvedLinkException, IOException {
+     IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       return rpcProxy.isFileClosed(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public Object getUnderlyingProxyObject() {
+     return rpcProxy;
+   }
+ 
+   @Override
+   public String createSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     final CreateSnapshotRequestProto.Builder builder
+         = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot);
+     if (snapshotName != null) {
+       builder.setSnapshotName(snapshotName);
+     }
+     final CreateSnapshotRequestProto req = builder.build();
+     try {
+       return rpcProxy.createSnapshot(null, req).getSnapshotPath();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public void deleteSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder()
+         .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build();
+     try {
+       rpcProxy.deleteSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public void allowSnapshot(String snapshotRoot) throws IOException {
+     AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
+         .setSnapshotRoot(snapshotRoot).build();
+     try {
+       rpcProxy.allowSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void disallowSnapshot(String snapshotRoot) throws IOException {
+     DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto
+         .newBuilder().setSnapshotRoot(snapshotRoot).build();
+     try {
+       rpcProxy.disallowSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+       String snapshotNewName) throws IOException {
+     RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder()
+         .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName)
+         .setSnapshotNewName(snapshotNewName).build();
+     try {
+       rpcProxy.renameSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+       throws IOException {
+     GetSnapshottableDirListingRequestProto req = 
+         GetSnapshottableDirListingRequestProto.newBuilder().build();
+     try {
+       GetSnapshottableDirListingResponseProto result = rpcProxy
+           .getSnapshottableDirListing(null, req);
+       
+       if (result.hasSnapshottableDirList()) {
+         return PBHelperClient.convert(result.getSnapshottableDirList());
+       }
+       return null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+       String fromSnapshot, String toSnapshot) throws IOException {
+     GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto
+         .newBuilder().setSnapshotRoot(snapshotRoot)
+         .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
+     try {
+       GetSnapshotDiffReportResponseProto result = 
+           rpcProxy.getSnapshotDiffReport(null, req);
+     
+       return PBHelperClient.convert(result.getDiffReport());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public long addCacheDirective(CacheDirectiveInfo directive,
+       EnumSet<CacheFlag> flags) throws IOException {
+     try {
+       AddCacheDirectiveRequestProto.Builder builder =
+           AddCacheDirectiveRequestProto.newBuilder().
+               setInfo(PBHelperClient.convert(directive));
+       if (!flags.isEmpty()) {
+         builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+       }
+       return rpcProxy.addCacheDirective(null, builder.build()).getId();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void modifyCacheDirective(CacheDirectiveInfo directive,
+       EnumSet<CacheFlag> flags) throws IOException {
+     try {
+       ModifyCacheDirectiveRequestProto.Builder builder =
+           ModifyCacheDirectiveRequestProto.newBuilder().
+               setInfo(PBHelperClient.convert(directive));
+       if (!flags.isEmpty()) {
+         builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+       }
+       rpcProxy.modifyCacheDirective(null, builder.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeCacheDirective(long id)
+       throws IOException {
+     try {
+       rpcProxy.removeCacheDirective(null,
+           RemoveCacheDirectiveRequestProto.newBuilder().
+               setId(id).build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   private static class BatchedCacheEntries
+       implements BatchedEntries<CacheDirectiveEntry> {
+     private final ListCacheDirectivesResponseProto response;
+ 
+     BatchedCacheEntries(
+         ListCacheDirectivesResponseProto response) {
+       this.response = response;
+     }
+ 
+     @Override
+     public CacheDirectiveEntry get(int i) {
+       return PBHelperClient.convert(response.getElements(i));
+     }
+ 
+     @Override
+     public int size() {
+       return response.getElementsCount();
+     }
+     
+     @Override
+     public boolean hasMore() {
+       return response.getHasMore();
+     }
+   }
+ 
+   @Override
+   public BatchedEntries<CacheDirectiveEntry>
+       listCacheDirectives(long prevId,
+           CacheDirectiveInfo filter) throws IOException {
+     if (filter == null) {
+       filter = new CacheDirectiveInfo.Builder().build();
+     }
+     try {
+       return new BatchedCacheEntries(
+         rpcProxy.listCacheDirectives(null,
+           ListCacheDirectivesRequestProto.newBuilder().
+             setPrevId(prevId).
+             setFilter(PBHelperClient.convert(filter)).
+             build()));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void addCachePool(CachePoolInfo info) throws IOException {
+     AddCachePoolRequestProto.Builder builder = 
+         AddCachePoolRequestProto.newBuilder();
+     builder.setInfo(PBHelperClient.convert(info));
+     try {
+       rpcProxy.addCachePool(null, builder.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void modifyCachePool(CachePoolInfo req) throws IOException {
+     ModifyCachePoolRequestProto.Builder builder = 
+         ModifyCachePoolRequestProto.newBuilder();
+     builder.setInfo(PBHelperClient.convert(req));
+     try {
+       rpcProxy.modifyCachePool(null, builder.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeCachePool(String cachePoolName) throws IOException {
+     try {
+       rpcProxy.removeCachePool(null, 
+           RemoveCachePoolRequestProto.newBuilder().
+             setPoolName(cachePoolName).build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   private static class BatchedCachePoolEntries
+     implements BatchedEntries<CachePoolEntry> {
+       private final ListCachePoolsResponseProto proto;
+     
+     public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
+       this.proto = proto;
+     }
+       
+     @Override
+     public CachePoolEntry get(int i) {
+       CachePoolEntryProto elem = proto.getEntries(i);
+       return PBHelperClient.convert(elem);
+     }
+ 
+     @Override
+     public int size() {
+       return proto.getEntriesCount();
+     }
+     
+     @Override
+     public boolean hasMore() {
+       return proto.getHasMore();
+     }
+   }
+ 
+   @Override
+   public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+       throws IOException {
+     try {
+       return new BatchedCachePoolEntries(
+         rpcProxy.listCachePools(null,
+           ListCachePoolsRequestProto.newBuilder().
+             setPrevPoolName(prevKey).build()));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException {
+     ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto
+         .newBuilder().setSrc(src)
+         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+     try {
+       rpcProxy.modifyAclEntries(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException {
+     RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto
+         .newBuilder().setSrc(src)
+         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+     try {
+       rpcProxy.removeAclEntries(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeDefaultAcl(String src) throws IOException {
+     RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto
+         .newBuilder().setSrc(src).build();
+     try {
+       rpcProxy.removeDefaultAcl(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeAcl(String src) throws IOException {
+     RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       rpcProxy.removeAcl(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+     SetAclRequestProto req = SetAclRequestProto.newBuilder()
+         .setSrc(src)
+         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
+         .build();
+     try {
+       rpcProxy.setAcl(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public AclStatus getAclStatus(String src) throws IOException {
+     GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void createEncryptionZone(String src, String keyName)
+     throws IOException {
+     final CreateEncryptionZoneRequestProto.Builder builder =
+       CreateEncryptionZoneRequestProto.newBuilder();
+     builder.setSrc(src);
+     if (keyName != null && !keyName.isEmpty()) {
+       builder.setKeyName(keyName);
+     }
+     CreateEncryptionZoneRequestProto req = builder.build();
+     try {
+       rpcProxy.createEncryptionZone(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public EncryptionZone getEZForPath(String src)
+       throws IOException {
+     final GetEZForPathRequestProto.Builder builder =
+         GetEZForPathRequestProto.newBuilder();
+     builder.setSrc(src);
+     final GetEZForPathRequestProto req = builder.build();
+     try {
+       final EncryptionZonesProtos.GetEZForPathResponseProto response =
+           rpcProxy.getEZForPath(null, req);
+       if (response.hasZone()) {
+         return PBHelperClient.convert(response.getZone());
+       } else {
+         return null;
+       }
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
+       throws IOException {
+     final ListEncryptionZonesRequestProto req =
+       ListEncryptionZonesRequestProto.newBuilder()
+           .setId(id)
+           .build();
+     try {
+       EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
+           rpcProxy.listEncryptionZones(null, req);
+       List<EncryptionZone> elements =
+           Lists.newArrayListWithCapacity(response.getZonesCount());
+       for (EncryptionZoneProto p : response.getZonesList()) {
+         elements.add(PBHelperClient.convert(p));
+       }
+       return new BatchedListEntries<EncryptionZone>(elements,
+           response.getHasMore());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
++  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
++      throws IOException {
++    final SetErasureCodingPolicyRequestProto.Builder builder =
++        SetErasureCodingPolicyRequestProto.newBuilder();
++    builder.setSrc(src);
++    if (ecPolicy != null) {
++      builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
++    }
++    SetErasureCodingPolicyRequestProto req = builder.build();
++    try {
++      rpcProxy.setErasureCodingPolicy(null, req);
++    } catch (ServiceException e) {
++      throw ProtobufHelper.getRemoteException(e);
++    }
++  }
++
++  @Override
+   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+       throws IOException {
+     SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
+         .setSrc(src)
+         .setXAttr(PBHelperClient.convertXAttrProto(xAttr))
+         .setFlag(PBHelperClient.convert(flag))
+         .build();
+     try {
+       rpcProxy.setXAttr(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+       throws IOException {
+     GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder();
+     builder.setSrc(src);
+     if (xAttrs != null) {
+       builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
+     }
+     GetXAttrsRequestProto req = builder.build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getXAttrs(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public List<XAttr> listXAttrs(String src)
+       throws IOException {
+     ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
+     builder.setSrc(src);
+     ListXAttrsRequestProto req = builder.build();
+     try {
+       return PBHelperClient.convert(rpcProxy.listXAttrs(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeXAttr(String src, XAttr xAttr) throws IOException {
+     RemoveXAttrRequestProto req = RemoveXAttrRequestProto
+         .newBuilder().setSrc(src)
+         .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build();
+     try {
+       rpcProxy.removeXAttr(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void checkAccess(String path, FsAction mode) throws IOException {
+     CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder()
+         .setPath(path).setMode(PBHelperClient.convert(mode)).build();
+     try {
+       rpcProxy.checkAccess(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setStoragePolicy(String src, String policyName)
+       throws IOException {
+     SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
+         .newBuilder().setSrc(src).setPolicyName(policyName).build();
+     try {
+       rpcProxy.setStoragePolicy(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+     GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto
+         .newBuilder().setPath(path).build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getStoragePolicy(null, request)
+           .getStoragePolicy());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+     try {
+       GetStoragePoliciesResponseProto response = rpcProxy
+           .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST);
+       return PBHelperClient.convertStoragePolicies(response.getPoliciesList());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   public long getCurrentEditLogTxid() throws IOException {
+     GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
+         .getDefaultInstance();
+     try {
+       return rpcProxy.getCurrentEditLogTxid(null, req).getTxid();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public EventBatchList getEditsFromTxid(long txid) throws IOException {
+     GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder()
+         .setTxid(txid).build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getEditsFromTxid(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
++
++  @Override
++  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
++    try {
++      GetErasureCodingPoliciesResponseProto response = rpcProxy
++          .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST);
++      ErasureCodingPolicy[] ecPolicies =
++          new ErasureCodingPolicy[response.getEcPoliciesCount()];
++      int i = 0;
++      for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) {
++        ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
++      }
++      return ecPolicies;
++    } catch (ServiceException e) {
++      throw ProtobufHelper.getRemoteException(e);
++    }
++  }
++
++  @Override
++  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
++    GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
++        .setSrc(src).build();
++    try {
++      GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
++          null, req);
++      if (response.hasEcPolicy()) {
++        return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy());
++      }
++      return null;
++    } catch (ServiceException e) {
++      throw ProtobufHelper.getRemoteException(e);
++    }
++  }
+ }


[23/58] [abbrv] hadoop git commit: HDFS-9148. Incorrect assert message in TestWriteToReplica#testWriteToTemporary (Tony Wu via Lei (Eddy) Xu)

Posted by zh...@apache.org.
HDFS-9148. Incorrect assert message in TestWriteToReplica#testWriteToTemporary (Tony Wu via Lei (Eddy) Xu)


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

Branch: refs/heads/HDFS-7285
Commit: 50741cb568d4da30b92d4954928bc3039e583b22
Parents: 4c9497c
Author: Lei Xu <le...@apache.org>
Authored: Mon Sep 28 13:23:19 2015 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Mon Sep 28 13:27:25 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java  | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50741cb5/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 1d9fa1d..5e1fd92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -988,6 +988,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9080. Update htrace version to 4.0.1 (cmccabe)
 
+    HDFS-9148. Incorrect assert message in TestWriteToReplica#testWriteToTemporary
+    (Tony Wu via lei)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50741cb5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
index a77184b..678654a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java
@@ -517,8 +517,8 @@ public class TestWriteToReplica {
       Assert.assertTrue(
           replicaInfo.getBlockId() == blocks[NON_EXISTENT].getBlockId());
     } catch (ReplicaAlreadyExistsException e) {
-      Assert.fail("createRbw() Should have removed the block with the older "
-          + "genstamp and replaced it with the newer one: " + blocks[NON_EXISTENT]);
+      Assert.fail("createTemporary should have allowed the block with newer "
+          + " generation stamp to be created " + blocks[NON_EXISTENT]);
     }
   }
   


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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();
+  }
+}


[18/58] [abbrv] hadoop git commit: HDFS-9080. Update htrace version to 4.0.1 (cmccabe)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
index a5a40f1..b2dc600 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
@@ -43,10 +43,12 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.tracing.TraceUtils;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.htrace.core.Tracer;
 import org.mortbay.util.ajax.JSON;
 
 import com.google.common.base.Preconditions;
@@ -69,6 +71,7 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
   private ObjectName journalNodeInfoBeanName;
   private String httpServerURI;
   private File localDir;
+  Tracer tracer;
 
   static {
     HdfsConfiguration.init();
@@ -105,6 +108,11 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
     this.localDir = new File(
         conf.get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
         DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_DEFAULT).trim());
+    if (this.tracer == null) {
+      this.tracer = new Tracer.Builder("JournalNode").
+          conf(TraceUtils.wrapHadoopConf("journalnode.htrace", conf)).
+          build();
+    }
   }
 
   private static void validateAndCreateJournalDir(File dir) throws IOException {
@@ -203,6 +211,10 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
       MBeans.unregister(journalNodeInfoBeanName);
       journalNodeInfoBeanName = null;
     }
+    if (tracer != null) {
+      tracer.close();
+      tracer = null;
+    }
   }
 
   /**
@@ -326,5 +338,4 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
   public Long getJournalCTime(String journalId) throws IOException {
     return getOrCreateJournal(journalId).getJournalCTime();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
index 16fe6b3..6c0783a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
@@ -85,6 +85,7 @@ class JournalNodeRpcServer implements QJournalProtocol {
       CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
           server.refreshServiceAcl(confCopy, new HDFSPolicyProvider());
     }
+    this.server.setTracer(jn.tracer);
   }
 
   void start() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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 d7e62bb..816eec4 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
@@ -47,9 +47,8 @@ import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.Sampler;
+import org.apache.htrace.core.TraceScope;
 
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_SEQUENTIAL;
@@ -708,8 +707,8 @@ class BlockSender implements java.io.Closeable {
    */
   long sendBlock(DataOutputStream out, OutputStream baseStream, 
                  DataTransferThrottler throttler) throws IOException {
-    TraceScope scope =
-        Trace.startSpan("sendBlock_" + block.getBlockId(), Sampler.NEVER);
+    TraceScope scope = datanode.tracer.
+        newScope("sendBlock_" + block.getBlockId());
     try {
       return doSendBlock(out, baseStream, throttler);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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 337706e..2646089 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
@@ -187,12 +187,13 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.tracing.SpanReceiverHost;
 import org.apache.hadoop.tracing.SpanReceiverInfo;
 import org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService;
 import org.apache.hadoop.tracing.TraceAdminProtocol;
 import org.apache.hadoop.tracing.TraceAdminProtocolPB;
 import org.apache.hadoop.tracing.TraceAdminProtocolServerSideTranslatorPB;
+import org.apache.hadoop.tracing.TraceUtils;
+import org.apache.hadoop.tracing.TracerConfigurationManager;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -202,6 +203,7 @@ import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
+import org.apache.htrace.core.Tracer;
 import org.mortbay.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -286,6 +288,8 @@ public class DataNode extends ReconfigurableBase
 
   public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog");
 
+  private static final String DATANODE_HTRACE_PREFIX = "datanode.htrace.";
+
   /**
    * Use {@link NetUtils#createSocketAddr(String)} instead.
    */
@@ -365,11 +369,18 @@ public class DataNode extends ReconfigurableBase
   private boolean isPermissionEnabled;
   private String dnUserName = null;
 
-  private SpanReceiverHost spanReceiverHost;
+  final Tracer tracer;
+  private final TracerConfigurationManager tracerConfigurationManager;
   private static final int NUM_CORES = Runtime.getRuntime()
       .availableProcessors();
   private static final double CONGESTION_RATIO = 1.5;
 
+  private static Tracer createTracer(Configuration conf) {
+    return new Tracer.Builder("DataNode").
+        conf(TraceUtils.wrapHadoopConf(DATANODE_HTRACE_PREFIX , conf)).
+        build();
+  }
+
   private long[] oobTimeouts; /** timeout value of each OOB type */
 
   private ScheduledThreadPoolExecutor metricsLoggerTimer;
@@ -381,6 +392,9 @@ public class DataNode extends ReconfigurableBase
   @InterfaceAudience.LimitedPrivate("HDFS")
   DataNode(final Configuration conf) {
     super(conf);
+    this.tracer = createTracer(conf);
+    this.tracerConfigurationManager =
+        new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
     this.fileDescriptorPassingDisabledReason = null;
     this.maxNumberOfBlocksToLog = 0;
     this.confVersion = null;
@@ -401,6 +415,9 @@ public class DataNode extends ReconfigurableBase
            final List<StorageLocation> dataDirs,
            final SecureResources resources) throws IOException {
     super(conf);
+    this.tracer = createTracer(conf);
+    this.tracerConfigurationManager =
+        new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
     this.blockScanner = new BlockScanner(this, conf);
     this.lastDiskErrorCheck = 0;
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
@@ -1137,9 +1154,6 @@ public class DataNode extends ReconfigurableBase
     this.dnConf = new DNConf(conf);
     checkSecureConfig(dnConf, conf, resources);
 
-    this.spanReceiverHost =
-      SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX);
-
     if (dnConf.maxLockedMemory > 0) {
       if (!NativeIO.POSIX.getCacheManipulator().verifyCanMlock()) {
         throw new RuntimeException(String.format(
@@ -1821,9 +1835,6 @@ public class DataNode extends ReconfigurableBase
       MBeans.unregister(dataNodeInfoBeanName);
       dataNodeInfoBeanName = null;
     }
-    if (this.spanReceiverHost != null) {
-      this.spanReceiverHost.closeReceivers();
-    }
     if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown();
     LOG.info("Shutdown complete.");
     synchronized(this) {
@@ -1832,6 +1843,7 @@ public class DataNode extends ReconfigurableBase
       // Notify the main thread.
       notifyAll();
     }
+    tracer.close();
   }
   
   
@@ -2264,6 +2276,7 @@ public class DataNode extends ReconfigurableBase
     if (localDataXceiverServer != null) {
       localDataXceiverServer.start();
     }
+    ipcServer.setTracer(tracer);
     ipcServer.start();
     startPlugins(conf);
   }
@@ -3262,19 +3275,19 @@ public class DataNode extends ReconfigurableBase
   @Override
   public SpanReceiverInfo[] listSpanReceivers() throws IOException {
     checkSuperuserPrivilege();
-    return spanReceiverHost.listSpanReceivers();
+    return tracerConfigurationManager.listSpanReceivers();
   }
 
   @Override
   public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
     checkSuperuserPrivilege();
-    return spanReceiverHost.addSpanReceiver(info);
+    return tracerConfigurationManager.addSpanReceiver(info);
   }
 
   @Override
   public void removeSpanReceiver(long id) throws IOException {
     checkSuperuserPrivilege();
-    spanReceiverHost.removeSpanReceiver(id);
+    tracerConfigurationManager.removeSpanReceiver(id);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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 665432e..b5d8f4f 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
@@ -126,7 +126,7 @@ class DataXceiver extends Receiver implements Runnable {
   
   private DataXceiver(Peer peer, DataNode datanode,
       DataXceiverServer dataXceiverServer) throws IOException {
-
+    super(datanode.tracer);
     this.peer = peer;
     this.dnConf = datanode.getDnConf();
     this.socketIn = peer.getInputStream();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 565555e..19c6415 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -71,13 +71,15 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.tracing.SpanReceiverHost;
 import org.apache.hadoop.tracing.TraceAdminProtocol;
+import org.apache.hadoop.tracing.TraceUtils;
+import org.apache.hadoop.tracing.TracerConfigurationManager;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.JvmPauseMonitor;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.htrace.core.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -318,6 +320,8 @@ public class NameNode implements NameNodeStatusMXBean {
   public static final HAState ACTIVE_STATE = new ActiveState();
   public static final HAState STANDBY_STATE = new StandbyState();
 
+  private static final String NAMENODE_HTRACE_PREFIX = "namenode.htrace.";
+
   public static final Log MetricsLog =
       LogFactory.getLog("NameNodeMetricsLog");
 
@@ -345,7 +349,8 @@ public class NameNode implements NameNodeStatusMXBean {
 
   private JvmPauseMonitor pauseMonitor;
   private ObjectName nameNodeStatusBeanName;
-  SpanReceiverHost spanReceiverHost;
+  protected final Tracer tracer;
+  protected final TracerConfigurationManager tracerConfigurationManager;
   ScheduledThreadPoolExecutor metricsLoggerTimer;
 
   /**
@@ -620,9 +625,6 @@ public class NameNode implements NameNodeStatusMXBean {
       startHttpServer(conf);
     }
 
-    this.spanReceiverHost =
-      SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX);
-
     loadNamesystem(conf);
 
     rpcServer = createRpcServer(conf);
@@ -810,8 +812,13 @@ public class NameNode implements NameNodeStatusMXBean {
     this(conf, NamenodeRole.NAMENODE);
   }
 
-  protected NameNode(Configuration conf, NamenodeRole role) 
-      throws IOException { 
+  protected NameNode(Configuration conf, NamenodeRole role)
+      throws IOException {
+    this.tracer = new Tracer.Builder("NameNode").
+        conf(TraceUtils.wrapHadoopConf(NAMENODE_HTRACE_PREFIX, conf)).
+        build();
+    this.tracerConfigurationManager =
+        new TracerConfigurationManager(NAMENODE_HTRACE_PREFIX, conf);
     this.conf = conf;
     this.role = role;
     setClientNamenodeAddress(conf);
@@ -894,10 +901,8 @@ public class NameNode implements NameNodeStatusMXBean {
         MBeans.unregister(nameNodeStatusBeanName);
         nameNodeStatusBeanName = null;
       }
-      if (this.spanReceiverHost != null) {
-        this.spanReceiverHost.closeReceivers();
-      }
     }
+    tracer.close();
   }
 
   synchronized boolean isStopRequested() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 6b7e8cf..79a3773 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -417,6 +417,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
         FSLimitException.PathComponentTooLongException.class,
         FSLimitException.MaxDirectoryItemsExceededException.class,
         UnresolvedPathException.class);
+    clientRpcServer.setTracer(nn.tracer);
+    if (serviceRpcServer != null) {
+      clientRpcServer.setTracer(nn.tracer);
+    }
  }
 
   /** Allow access to the client RPC server for testing */
@@ -2025,20 +2029,20 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public SpanReceiverInfo[] listSpanReceivers() throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    return nn.spanReceiverHost.listSpanReceivers();
+    return nn.tracerConfigurationManager.listSpanReceivers();
   }
 
   @Override // TraceAdminProtocol
   public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    return nn.spanReceiverHost.addSpanReceiver(info);
+    return nn.tracerConfigurationManager.addSpanReceiver(info);
   }
 
   @Override // TraceAdminProtocol
   public void removeSpanReceiver(long id) throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    nn.spanReceiverHost.removeSpanReceiver(id);
+    nn.tracerConfigurationManager.removeSpanReceiver(id);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 2dbc15a..6d40898 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -81,6 +81,7 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Time;
+import org.apache.htrace.core.Tracer;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -137,6 +138,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
   private boolean showReplicaDetails = false;
   private long staleInterval;
+  private Tracer tracer;
+
   /**
    * True if we encountered an internal error during FSCK, such as not being
    * able to delete a corrupt file.
@@ -199,6 +202,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     this.staleInterval =
         conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
+    this.tracer = new Tracer.Builder("NamenodeFsck").build();
 
     for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
       String key = it.next();
@@ -874,6 +878,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
             setCachingStrategy(CachingStrategy.newDropBehind()).
             setClientCacheContext(dfs.getClientContext()).
             setConfiguration(namenode.conf).
+            setTracer(tracer).
             setRemotePeerFactory(new RemotePeerFactory() {
               @Override
               public Peer newConnectedPeer(InetSocketAddress addr,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 0c1ad7d..f1b855e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2428,19 +2428,10 @@
 </property>
 
 <property>
-  <name>dfs.htrace.spanreceiver.classes</name>
+  <name>dfs.client.htrace.sampler.classes</name>
   <value></value>
   <description>
-    The class name of the HTrace SpanReceiver for the NameNode and DataNode.
-  </description>
-</property>
-
-<property>
-  <name>dfs.client.htrace.spanreceiver.classes</name>
-  <value></value>
-  <description>
-    The class name of the HTrace SpanReceiver for the HDFS client. You do not
-    need to enable this if your client program has been modified to use HTrace.
+    The class names of the HTrace Samplers to use for the HDFS client.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
index 7b0ef02..ba25d97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
@@ -30,6 +30,7 @@ import java.util.Random;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -192,6 +193,7 @@ public class BlockReaderTestUtil {
       setCachingStrategy(CachingStrategy.newDefaultStrategy()).
       setConfiguration(fs.getConf()).
       setAllowShortCircuitLocalReads(true).
+      setTracer(FsTracer.get(fs.getConf())).
       setRemotePeerFactory(new RemotePeerFactory() {
         @Override
         public Peer newConnectedPeer(InetSocketAddress addr,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
index 1c43948..2d6c63a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -195,6 +196,7 @@ public class TestBlockReaderLocal {
           setShortCircuitReplica(replica).
           setCachingStrategy(new CachingStrategy(false, readahead)).
           setVerifyChecksum(checksum).
+          setTracer(FsTracer.get(conf)).
           build();
       dataIn = null;
       metaIn = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index e627455..821d259 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@@ -34,6 +35,7 @@ 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.datatransfer.BlockConstructionStage;
+import org.apache.htrace.core.SpanId;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -119,6 +121,7 @@ public class TestDFSOutputStream {
     DfsClientConf dfsClientConf = mock(DfsClientConf.class);
     DFSClient client = mock(DFSClient.class);
     when(client.getConf()).thenReturn(dfsClientConf);
+    when(client.getTracer()).thenReturn(FsTracer.get(new Configuration()));
     client.clientRunning = true;
     DataStreamer stream = new DataStreamer(
         mock(HdfsFileStatus.class),
@@ -139,7 +142,7 @@ public class TestDFSOutputStream {
         Whitebox.getInternalState(stream, "congestedNodes");
     congestedNodes.add(mock(DatanodeInfo.class));
     DFSPacket packet = mock(DFSPacket.class);
-    when(packet.getTraceParents()).thenReturn(new long[] {});
+    when(packet.getTraceParents()).thenReturn(new SpanId[] {});
     dataQueue.add(packet);
     stream.run();
     Assert.assertTrue(congestedNodes.isEmpty());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPacket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPacket.java
index daee608..77957bc 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPacket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPacket.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import java.util.Random;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.htrace.core.SpanId;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -70,24 +71,24 @@ public class TestDFSPacket {
   public void testAddParentsGetParents() throws Exception {
     DFSPacket p = new DFSPacket(null, maxChunksPerPacket,
                                 0, 0, checksumSize, false);
-    long parents[] = p.getTraceParents();
+    SpanId parents[] = p.getTraceParents();
     Assert.assertEquals(0, parents.length);
-    p.addTraceParent(123);
-    p.addTraceParent(123);
+    p.addTraceParent(new SpanId(0, 123));
+    p.addTraceParent(new SpanId(0, 123));
     parents = p.getTraceParents();
     Assert.assertEquals(1, parents.length);
-    Assert.assertEquals(123, parents[0]);
+    Assert.assertEquals(new SpanId(0, 123), parents[0]);
     parents = p.getTraceParents(); // test calling 'get' again.
     Assert.assertEquals(1, parents.length);
-    Assert.assertEquals(123, parents[0]);
-    p.addTraceParent(1);
-    p.addTraceParent(456);
-    p.addTraceParent(789);
+    Assert.assertEquals(new SpanId(0, 123), parents[0]);
+    p.addTraceParent(new SpanId(0, 1));
+    p.addTraceParent(new SpanId(0, 456));
+    p.addTraceParent(new SpanId(0, 789));
     parents = p.getTraceParents();
     Assert.assertEquals(4, parents.length);
-    Assert.assertEquals(1, parents[0]);
-    Assert.assertEquals(123, parents[1]);
-    Assert.assertEquals(456, parents[2]);
-    Assert.assertEquals(789, parents[3]);
+    Assert.assertEquals(new SpanId(0, 1), parents[0]);
+    Assert.assertEquals(new SpanId(0, 123), parents[1]);
+    Assert.assertEquals(new SpanId(0, 456), parents[2]);
+    Assert.assertEquals(new SpanId(0, 789), parents[3]);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
index cf00405..5977dbf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -88,7 +89,6 @@ import org.apache.hadoop.util.GSet;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.htrace.Sampler;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -967,7 +967,7 @@ public class TestCacheDirectives {
 
     // Uncache and check each path in sequence
     RemoteIterator<CacheDirectiveEntry> entries =
-      new CacheDirectiveIterator(nnRpc, null, Sampler.NEVER);
+      new CacheDirectiveIterator(nnRpc, null, FsTracer.get(conf));
     for (int i=0; i<numFiles; i++) {
       CacheDirectiveEntry entry = entries.next();
       nnRpc.removeCacheDirective(entry.getInfo().getId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 179d50b..a84ddd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -398,6 +398,7 @@ public class TestFsck {
 
       // Copy the non-corrupt blocks of corruptFileName to lost+found.
       outStr = runFsck(conf, 1, false, "/", "-move");
+      FSImage.LOG.info("WATERMELON: outStr = " + outStr);
       assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
 
       // Make sure that we properly copied the block files from the DataNodes
@@ -1656,4 +1657,4 @@ public class TestFsck {
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
index ec0450a..1c029fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
@@ -77,8 +77,8 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
     // Some properties have moved to HdfsClientConfigKeys
     xmlPropsToSkipCompare.add("dfs.client.short.circuit.replica.stale.threshold.ms");
 
-    // Ignore SpanReceiveHost properties
-    xmlPropsToSkipCompare.add("dfs.htrace.spanreceiver.classes");
-    xmlPropsToSkipCompare.add("dfs.client.htrace.spanreceiver.classes");
+    // Ignore HTrace properties
+    xmlPropsToSkipCompare.add("fs.client.htrace");
+    xmlPropsToSkipCompare.add("hadoop.htrace");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
index acd0dbb..b08866b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.apache.htrace.core.Tracer;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -58,8 +59,8 @@ public class TestTraceAdmin {
   public void testCreateAndDestroySpanReceiver() throws Exception {
     Configuration conf = new Configuration();
     conf = new Configuration();
-    conf.set(DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX  +
-        SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX, "");
+    conf.set(TraceUtils.DEFAULT_HADOOP_PREFIX +
+        Tracer.SPAN_RECEIVER_CLASSES_KEY, "");
     MiniDFSCluster cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
@@ -74,12 +75,12 @@ public class TestTraceAdmin {
       Assert.assertEquals("ret:0, Added trace span receiver 1 with " +
           "configuration dfs.htrace.local-file-span-receiver.path = " + tracePath + NEWLINE,
           runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
-              "-class", "org.apache.htrace.impl.LocalFileSpanReceiver",
+              "-class", "org.apache.htrace.core.LocalFileSpanReceiver",
               "-Cdfs.htrace.local-file-span-receiver.path=" + tracePath));
       String list =
           runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster));
       Assert.assertTrue(list.startsWith("ret:0"));
-      Assert.assertTrue(list.contains("1   org.apache.htrace.impl.LocalFileSpanReceiver"));
+      Assert.assertTrue(list.contains("1   org.apache.htrace.core.LocalFileSpanReceiver"));
       Assert.assertEquals("ret:0, Removed trace span receiver 1" + NEWLINE,
           runTraceCommand(trace, "-remove", "1", "-host",
               getHostPortForNN(cluster)));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
index 5b365ba..7d71441 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
@@ -17,50 +17,72 @@
  */
 package org.apache.hadoop.tracing;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.Sampler;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
 
 public class TestTracing {
-
-  private static Configuration conf;
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem dfs;
 
+  private Tracer prevTracer;
+
+  private final static Configuration TRACING_CONF;
+  private final static Configuration NO_TRACING_CONF;
+
+  static {
+    NO_TRACING_CONF = new Configuration();
+    NO_TRACING_CONF.setLong("dfs.blocksize", 100 * 1024);
+
+    TRACING_CONF = new Configuration(NO_TRACING_CONF);
+    TRACING_CONF.set(CommonConfigurationKeys.FS_CLIENT_HTRACE_PREFIX +
+        Tracer.SPAN_RECEIVER_CLASSES_KEY,
+        SetSpanReceiver.class.getName());
+    TRACING_CONF.set(CommonConfigurationKeys.FS_CLIENT_HTRACE_PREFIX +
+        Tracer.SAMPLER_CLASSES_KEY, "AlwaysSampler");
+  }
+
   @Test
   public void testTracing() throws Exception {
     // write and read without tracing started
     String fileName = "testTracingDisabled.dat";
     writeTestFile(fileName);
-    Assert.assertTrue(SetSpanReceiver.size() == 0);
+    Assert.assertEquals(0, SetSpanReceiver.size());
     readTestFile(fileName);
-    Assert.assertTrue(SetSpanReceiver.size() == 0);
+    Assert.assertEquals(0, SetSpanReceiver.size());
 
-    writeWithTracing();
-    readWithTracing();
+    writeTestFile("testReadTraceHooks.dat");
+
+    FsTracer.clear();
+    Tracer tracer = FsTracer.get(TRACING_CONF);
+    writeWithTracing(tracer);
+    readWithTracing(tracer);
   }
 
-  public void writeWithTracing() throws Exception {
+  private void writeWithTracing(Tracer tracer) throws Exception {
     long startTime = System.currentTimeMillis();
-    TraceScope ts = Trace.startSpan("testWriteTraceHooks", Sampler.ALWAYS);
+    TraceScope ts = tracer.newScope("testWriteTraceHooks");
     writeTestFile("testWriteTraceHooks.dat");
     long endTime = System.currentTimeMillis();
     ts.close();
@@ -107,7 +129,8 @@ public class TestTracing {
     };
     for (String desc : spansInTopTrace) {
       for (Span span : map.get(desc)) {
-        Assert.assertEquals(ts.getSpan().getTraceId(), span.getTraceId());
+        Assert.assertEquals(ts.getSpan().getSpanId().getHigh(),
+                            span.getSpanId().getHigh());
       }
     }
 
@@ -120,12 +143,10 @@ public class TestTracing {
     SetSpanReceiver.clear();
   }
 
-  public void readWithTracing() throws Exception {
-    String fileName = "testReadTraceHooks.dat";
-    writeTestFile(fileName);
+  private void readWithTracing(Tracer tracer) throws Exception {
     long startTime = System.currentTimeMillis();
-    TraceScope ts = Trace.startSpan("testReadTraceHooks", Sampler.ALWAYS);
-    readTestFile(fileName);
+    TraceScope ts = tracer.newScope("testReadTraceHooks");
+    readTestFile("testReadTraceHooks.dat");
     ts.close();
     long endTime = System.currentTimeMillis();
 
@@ -150,7 +171,11 @@ public class TestTracing {
     // There should only be one trace id as it should all be homed in the
     // top trace.
     for (Span span : SetSpanReceiver.getSpans()) {
-      Assert.assertEquals(ts.getSpan().getTraceId(), span.getTraceId());
+      System.out.println(span.toJson());
+    }
+    for (Span span : SetSpanReceiver.getSpans()) {
+      Assert.assertEquals(ts.getSpan().getSpanId().getHigh(),
+                          span.getSpanId().getHigh());
     }
     SetSpanReceiver.clear();
   }
@@ -185,18 +210,9 @@ public class TestTracing {
     }
   }
 
-  @BeforeClass
-  public static void setup() throws IOException {
-    conf = new Configuration();
-    conf.setLong("dfs.blocksize", 100 * 1024);
-    conf.set(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
-        SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
-        SetSpanReceiver.class.getName());
-  }
-
   @Before
   public void startCluster() throws IOException {
-    cluster = new MiniDFSCluster.Builder(conf)
+    cluster = new MiniDFSCluster.Builder(NO_TRACING_CONF)
         .numDataNodes(3)
         .build();
     cluster.waitActive();
@@ -207,6 +223,6 @@ public class TestTracing {
   @After
   public void shutDown() throws IOException {
     cluster.shutdown();
+    FsTracer.clear();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
index a34748d..292c68d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -33,9 +35,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.apache.hadoop.util.NativeCodeLoader;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,7 +45,6 @@ public class TestTracingShortCircuitLocalRead {
   private static Configuration conf;
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem dfs;
-  private static SpanReceiverHost spanReceiverHost;
   private static TemporarySocketDirectory sockDir;
   static final Path TEST_PATH = new Path("testShortCircuitTraceHooks");
   static final int TEST_LENGTH = 1234;
@@ -64,9 +64,12 @@ public class TestTracingShortCircuitLocalRead {
   public void testShortCircuitTraceHooks() throws IOException {
     assumeTrue(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS);
     conf = new Configuration();
-    conf.set(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
-        SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
+    conf.set(TraceUtils.DEFAULT_HADOOP_PREFIX +
+            Tracer.SPAN_RECEIVER_CLASSES_KEY,
         SetSpanReceiver.class.getName());
+    conf.set(TraceUtils.DEFAULT_HADOOP_PREFIX +
+            Tracer.SAMPLER_CLASSES_KEY,
+        "AlwaysSampler");
     conf.setLong("dfs.blocksize", 100 * 1024);
     conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
     conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
@@ -81,7 +84,8 @@ public class TestTracingShortCircuitLocalRead {
     try {
       DFSTestUtil.createFile(dfs, TEST_PATH, TEST_LENGTH, (short)1, 5678L);
 
-      TraceScope ts = Trace.startSpan("testShortCircuitTraceHooks", Sampler.ALWAYS);
+      TraceScope ts = FsTracer.get(conf).
+          newScope("testShortCircuitTraceHooks");
       FSDataInputStream stream = dfs.open(TEST_PATH);
       byte buf[] = new byte[TEST_LENGTH];
       IOUtils.readFully(stream, buf, 0, TEST_LENGTH);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 293c279..fd33246 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -807,8 +807,8 @@
       </dependency>
       <dependency>
         <groupId>org.apache.htrace</groupId>
-        <artifactId>htrace-core</artifactId>
-        <version>3.2.0-incubating</version>
+        <artifactId>htrace-core4</artifactId>
+        <version>4.0.1-incubating</version>
       </dependency>
       <dependency>
         <groupId>org.jdom</groupId>


[29/58] [abbrv] hadoop git commit: HADOOP-12440. TestRPC#testRPCServerShutdown did not produce the desired thread states before shutting down. (Xiao Chen via mingma)

Posted by zh...@apache.org.
HADOOP-12440. TestRPC#testRPCServerShutdown did not produce the desired thread states before shutting down. (Xiao Chen via mingma)


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

Branch: refs/heads/HDFS-7285
Commit: 5c3b663bf95551d1cf36a2a39849e0676893fa1d
Parents: 9735afe
Author: Ming Ma <mi...@apache.org>
Authored: Mon Sep 28 18:12:51 2015 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Mon Sep 28 18:12:51 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 .../src/test/java/org/apache/hadoop/ipc/TestRPC.java             | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c3b663b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 07463f4..2af6580 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1095,6 +1095,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11918. Listing an empty s3a root directory throws FileNotFound.
     (Lei (Eddy) Xu via cnauroth)
 
+    HADOOP-12440. TestRPC#testRPCServerShutdown did not produce the desired
+    thread states before shutting down. (Xiao Chen via mingma)
+
   OPTIMIZATIONS
 
     HADOOP-12051. ProtobufRpcEngine.invoke() should use Exception.toString()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c3b663b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index d36a671..5711587 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -1060,8 +1060,8 @@ public class TestRPC {
             }));
       }
       while (server.getCallQueueLen() != 1
-          && countThreads(CallQueueManager.class.getName()) != 1
-          && countThreads(TestProtocol.class.getName()) != 1) {
+          || countThreads(CallQueueManager.class.getName()) != 1
+          || countThreads(TestImpl.class.getName()) != 1) {
         Thread.sleep(100);
       }
     } finally {


[47/58] [abbrv] hadoop git commit: HDFS-9174. Fix findbugs warnings in FSOutputSummer.tracer and DirectoryScanner$ReportCompiler.currentThread. Contributed by Yi Liu.

Posted by zh...@apache.org.
HDFS-9174. Fix findbugs warnings in FSOutputSummer.tracer and DirectoryScanner$ReportCompiler.currentThread. Contributed by Yi Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 8703301b466cbc37ef53a96a55bcf6412792d5cf
Parents: 80d33b5
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 29 14:56:06 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 29 14:56:06 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/FSOutputSummer.java   | 3 ---
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 .../apache/hadoop/hdfs/server/datanode/DirectoryScanner.java | 8 --------
 3 files changed, 3 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8703301b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index 648043e..68ff557 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -43,8 +42,6 @@ abstract public class FSOutputSummer extends OutputStream {
   private byte checksum[];
   // The number of valid bytes in the buffer.
   private int count;
-  // The HTrace tracer to use
-  private Tracer tracer;
   
   // We want this value to be a multiple of 3 because the native code checksums
   // 3 chunks simultaneously. The chosen value of 9 strikes a balance between

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8703301b/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 0876727..0bca0e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1459,6 +1459,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9141. Thread leak in Datanode#refreshVolumes. (Uma Maheswara Rao G
     via yliu)
 
+    HDFS-9174. Fix findbugs warnings in FSOutputSummer.tracer and
+    DirectoryScanner$ReportCompiler.currentThread. (Yi Liu via wheat9)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8703301b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index b8ea5bf..392c121 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -785,12 +785,6 @@ public class DirectoryScanner implements Runnable {
     private final StopWatch perfTimer = new StopWatch();
 
     /**
-     * The associated thread.  Used for testing purposes only.
-     */
-    @VisibleForTesting
-    Thread currentThread;
-
-    /**
      * Create a report compiler for the given volume on the given datanode.
      *
      * @param datanode the target datanode
@@ -809,8 +803,6 @@ public class DirectoryScanner implements Runnable {
      */
     @Override
     public ScanInfoPerBlockPool call() throws IOException {
-      currentThread = Thread.currentThread();
-
       String[] bpList = volume.getBlockPoolList();
       ScanInfoPerBlockPool result = new ScanInfoPerBlockPool(bpList.length);
       for (String bpid : bpList) {


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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/bf37d3d8/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 3bad9d2..f289b32 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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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();
-  }
-}


[49/58] [abbrv] hadoop git commit: HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index e245d2a..e122748 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -233,16 +233,12 @@ public class WebHdfsFileSystem extends FileSystem
       // refetch tokens.  even if ugi has credentials, don't attempt
       // to get another token to match hdfs/rpc behavior
       if (token != null) {
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("Using UGI token: {}", token);
-        }
+        LOG.debug("Using UGI token: {}", token);
         canRefreshDelegationToken = false;
       } else {
         token = getDelegationToken(null);
         if (token != null) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Fetched new token: {}", token);
-          }
+          LOG.debug("Fetched new token: {}", token);
         } else { // security is disabled
           canRefreshDelegationToken = false;
         }
@@ -257,9 +253,7 @@ public class WebHdfsFileSystem extends FileSystem
     boolean replaced = false;
     if (canRefreshDelegationToken) {
       Token<?> token = getDelegationToken(null);
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Replaced expired token: {}", token);
-      }
+      LOG.debug("Replaced expired token: {}", token);
       setDelegationToken(token);
       replaced = (token != null);
     }
@@ -442,9 +436,7 @@ public class WebHdfsFileSystem extends FileSystem
     InetSocketAddress nnAddr = getCurrentNNAddr();
     final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("url={}", url);
-    }
+    LOG.trace("url={}", url);
     return url;
   }
 
@@ -479,9 +471,7 @@ public class WebHdfsFileSystem extends FileSystem
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", parameters);
     final URL url = getNamenodeURL(path, query);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("url={}", url);
-    }
+    LOG.trace("url={}", url);
     return url;
   }
 
@@ -769,9 +759,7 @@ public class WebHdfsFileSystem extends FileSystem
       } catch (Exception e) { // catch json parser errors
         final IOException ioe =
             new IOException("Response decoding failure: "+e.toString(), e);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Response decoding failure: {}", e.toString(), e);
-        }
+        LOG.debug("Response decoding failure.", e);
         throw ioe;
       } finally {
         conn.disconnect();
@@ -1242,9 +1230,7 @@ public class WebHdfsFileSystem extends FileSystem
         cancelDelegationToken(delegationToken);
       }
     } catch (IOException ioe) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Token cancel failed: ", ioe);
-      }
+      LOG.debug("Token cancel failed: ", ioe);
     } finally {
       super.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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 7b62b97..dfd0b57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1000,6 +1000,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8696. Make the lower and higher watermark in the DN Netty server
     configurable. (Xiaobing Zhou via wheat9)
 
+    HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client
+    package. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[36/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 0000000,d1d8d37..af7a61e
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,1947 +1,1944 @@@
+ /**
+  * 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.FileSystem;
+ 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.core.Sampler;
+ import org.apache.htrace.core.Span;
+ import org.apache.htrace.core.SpanId;
+ import org.apache.htrace.core.TraceScope;
+ import org.apache.htrace.core.Tracer;
+ 
+ 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 {
++  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));
++          LOG.trace("Got Exception while checking, " + DataStreamer.this,
++              new Throwable(thrown));
+         }
+         final IOException e = thrown;
+         if (resetToNull) {
+           thrown = null;
+         }
+         throw e;
+       }
+     }
+ 
+     synchronized void throwException4Close() throws IOException {
+       check(false);
+       throw new ClosedChannelException();
+     }
+   }
+ 
++  enum ErrorType {
++    NONE, INTERNAL, EXTERNAL
++  }
++
+   static class ErrorState {
 -    private boolean error = false;
++    ErrorType error = ErrorType.NONE;
+     private int badNodeIndex = -1;
+     private int restartingNodeIndex = -1;
+     private long restartingNodeDeadline = 0;
+     private final long datanodeRestartTimeout;
+ 
+     ErrorState(long datanodeRestartTimeout) {
+       this.datanodeRestartTimeout = datanodeRestartTimeout;
+     }
+ 
++    synchronized void resetInternalError() {
++      if (hasInternalError()) {
++        error = ErrorType.NONE;
++      }
++      badNodeIndex = -1;
++      restartingNodeIndex = -1;
++      restartingNodeDeadline = 0;
++    }
++
+     synchronized void reset() {
 -      error = false;
++      error = ErrorType.NONE;
+       badNodeIndex = -1;
+       restartingNodeIndex = -1;
+       restartingNodeDeadline = 0;
+     }
+ 
++    synchronized boolean hasInternalError() {
++      return error == ErrorType.INTERNAL;
++    }
++
++    synchronized boolean hasExternalError() {
++      return error == ErrorType.EXTERNAL;
++    }
++
+     synchronized boolean hasError() {
 -      return error;
++      return error != ErrorType.NONE;
+     }
+ 
+     synchronized boolean hasDatanodeError() {
 -      return error && isNodeMarked();
++      return error == ErrorType.INTERNAL && isNodeMarked();
+     }
+ 
 -    synchronized void setError(boolean err) {
 -      this.error = err;
++    synchronized void setInternalError() {
++      this.error = ErrorType.INTERNAL;
++    }
++
++    synchronized void setExternalError() {
++      if (!hasInternalError()) {
++        this.error = ErrorType.EXTERNAL;
++      }
+     }
+ 
+     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;
++        error = ErrorType.NONE;
+       }
+       badNodeIndex = -1;
+     }
+ 
+     synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
+       if (restartingNodeIndex >= 0) {
 -        if (!error) {
++        if (error == ErrorType.NONE) {
+           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;
++  protected ExtendedBlock block; // its length is number of bytes acked
++  protected 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
++  protected 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;
++  protected final DFSClient dfsClient;
++  protected final String src;
+   /** Only for DataTransferProtocol.writeBlock(..) */
 -  private final DataChecksum checksum4WriteBlock;
 -  private final Progressable progress;
 -  private final HdfsFileStatus stat;
++  final DataChecksum checksum4WriteBlock;
++  final Progressable progress;
++  protected 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<>();
++  protected 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;
++  protected final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+   private final String[] favoredNodes;
+ 
 -  private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
++  private DataStreamer(HdfsFileStatus stat, ExtendedBlock block,
++                       DFSClient dfsClient, String src,
+                        Progressable progress, DataChecksum checksum,
+                        AtomicReference<CachingStrategy> cachingStrategy,
+                        ByteArrayManager byteArrayManage,
+                        boolean isAppend, String[] favoredNodes) {
++    this.block = block;
+     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,
++    this(stat, block, 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,
++    this(stat, lastBlock.getBlock(), 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);
+     }
+   }
+ 
++  void setAccessToken(Token<BlockTokenIdentifier> t) {
++    this.accessToken = t;
++  }
++
+   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() {
++  protected 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 = null;
+     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();
++        boolean doSleep = processDatanodeOrExternalError();
+ 
+         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
+             SpanId[] parents = one.getTraceParents();
+             if (parents.length > 0) {
+               scope = dfsClient.getTracer().
+                   newScope("dataStreamer", parents[0]);
+               scope.getSpan().setParents(parents);
+             }
+           }
+         }
+ 
+         // get new block from namenode.
++        if (LOG.isDebugEnabled()) {
++          LOG.debug("stage=" + stage + ", " + this);
++        }
+         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);
++              " < lastByteOffsetInBlock, " + this + ", " + one);
+         }
+ 
+         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
+         SpanId spanId = SpanId.INVALID;
+         synchronized (dataQueue) {
+           // move packet from dataQueue to ackQueue
+           if (!one.isHeartbeatPacket()) {
+             if (scope != null) {
+               spanId = scope.getSpanId();
+               scope.detach();
+               one.setTraceScope(scope);
+             }
+             scope = null;
+             dataQueue.removeFirst();
+             ackQueue.addLast(one);
+             dataQueue.notifyAll();
+           }
+         }
+ 
+         if (LOG.isDebugEnabled()) {
 -          LOG.debug("DataStreamer block " + block +
 -              " sending packet " + one);
++          LOG.debug(this + " sending " + one);
+         }
+ 
+         // write out data to remote datanode
+         TraceScope writeScope = dfsClient.getTracer().
+             newScope("DataStreamer#writeTo", spanId);
+         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);
++        errorState.setInternalError();
+         if (!errorState.isNodeMarked()) {
+           // Not a datanode issue
+           streamerClosed = true;
+         }
+       } finally {
+         if (scope != null) {
+           scope.close();
+           scope = null;
+         }
+       }
+     }
+     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 = dfsClient.getTracer().
+         newScope("waitForAckedSeqno");
+     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 = Tracer.getCurrentSpan();
+               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 = Tracer.getCurrentSpan();
+           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();
+     }
+   }
+ 
++  void setStreamerAsClosed() {
++    streamerClosed = true;
++  }
+ 
+   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() {
++  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 = null;
+       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 = one.getTraceScope();
+             if (scope != null) {
+               scope.reattach();
+               one.setTraceScope(null);
+             }
+             lastAckedSeqno = seqno;
+             ackQueue.removeFirst();
+             dataQueue.notifyAll();
+ 
+             one.releaseBuffer(byteArrayManager);
+           }
+         } catch (Exception e) {
+           if (!responderClosed) {
+             lastException.set(e);
 -            errorState.setError(true);
++            errorState.setInternalError();
+             errorState.markFirstNodeIfNotMarked();
+             synchronized (dataQueue) {
+               dataQueue.notifyAll();
+             }
+             if (!errorState.isRestartingNode()) {
+               LOG.warn("Exception for " + block, e);
+             }
+             responderClosed = true;
+           }
+         } finally {
+           if (scope != null) {
+             scope.close();
+           }
+           scope = null;
+         }
+       }
+     }
+ 
+     void close() {
+       responderClosed = true;
+       this.interrupt();
+     }
+   }
+ 
++  private boolean shouldHandleExternalError(){
++    return errorState.hasExternalError() && blockStream != null;
++  }
++
+   /**
+    * 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()) {
++  private boolean processDatanodeOrExternalError() throws IOException {
++    if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) {
+       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();
++
++    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
+           // Close any trace span associated with this Packet
+           TraceScope scope = endOfBlockPacket.getTraceScope();
+           if (scope != null) {
+             scope.reattach();
+             scope.close();
+             endOfBlockPacket.setTraceScope(null);
+           }
+           assert endOfBlockPacket.isLastPacketInBlock();
+           assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
+           lastAckedSeqno = endOfBlockPacket.getSeqno();
+           dataQueue.notifyAll();
+         }
+         endBlock();
+       } else {
+         initDataStreaming();
+       }
+     }
+ 
 -    return doSleep;
++    return false;
+   }
+ 
+   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;
+     }
+ 
+     int tried = 0;
+     final DatanodeInfo[] original = nodes;
+     final StorageType[] originalTypes = storageTypes;
+     final String[] originalIDs = storageIDs;
+     IOException caughtException = null;
+     ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
+     while (tried < 3) {
+       LocatedBlock lb;
+       //get a new datanode
+       lb = dfsClient.namenode.getAdditionalDatanode(
+           src, stat.getFileId(), block, nodes, storageIDs,
+           exclude.toArray(new DatanodeInfo[exclude.size()]),
+           1, dfsClient.clientName);
+       // a new node was allocated by the namenode. Update nodes.
+       setPipeline(lb);
+ 
+       //find the new datanode
+       final int d = findNewDatanode(original);
+       //transfer replica. pick a source from the original nodes
+       final DatanodeInfo src = original[tried % original.length];
+       final DatanodeInfo[] targets = {nodes[d]};
+       final StorageType[] targetStorageTypes = {storageTypes[d]};
+ 
+       try {
+         transfer(src, targets, targetStorageTypes, lb.getBlockToken());
+       } catch (IOException ioe) {
+         DFSClient.LOG.warn("Error transferring data from " + src + " to " +
+             nodes[d] + ": " + ioe.getMessage());
+         caughtException = ioe;
+         // add the allocated node to the exclude list.
+         exclude.add(nodes[d]);
+         setPipeline(original, originalTypes, originalIDs);
+         tried++;
+         continue;
+       }
+       return; // finished successfully
+     }
+     // All retries failed
+     throw (caughtException != null) ? caughtException :
+         new IOException("Failed to add a node");
+   }
+ 
+   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);
+ 
+       // transfer timeout multiplier based on the transfer size
+       // One per 200 packets = 12.8MB. Minimum is 2.
+       int multi = 2 + (int)(bytesSent/dfsClient.getConf().getWritePacketSize())/200;
+       final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
+ 
+       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 {
++  private void 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;
++      return;
+     }
++    setupPipelineInternal(nodes, storageTypes);
++  }
+ 
++  protected void setupPipelineInternal(DatanodeInfo[] datanodes,
++      StorageType[] nodeStorageTypes) throws IOException {
+     boolean success = false;
+     long newGS = 0L;
+     while (!success && !streamerClosed && dfsClient.clientRunning) {
+       if (!handleRestartingDatanode()) {
 -        return false;
++        return;
+       }
+ 
 -      final boolean isRecovery = errorState.hasError();
++      final boolean isRecovery = errorState.hasInternalError();
+       if (!handleBadDatanode()) {
 -        return false;
++        return;
+       }
+ 
+       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() {
++  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() {
++  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() {
++  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);
++  private LocatedBlock updateBlockForPipeline() throws IOException {
++    return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
++  }
++
++  static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
++    return new ExtendedBlock(b.getBlockPoolId(), b.getBlockId(),
++        b.getNumBytes(), newGS);
+   }
+ 
+   /** update pipeline at the namenode */
+   ExtendedBlock updatePipeline(long newGS) throws IOException {
 -    final ExtendedBlock newBlock = new ExtendedBlock(
 -        block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
++    final ExtendedBlock newBlock = newBlock(block, newGS);
+     dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+         nodes, storageIDs);
+     return newBlock;
+   }
+ 
++  private int getNumBlockWriteRetry() {
++    return dfsClient.getConf().getNumBlockWriteRetry();
++  }
++
+   /**
+    * 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 {
++  protected LocatedBlock nextBlockOutputStream() throws IOException {
+     LocatedBlock lb = null;
+     DatanodeInfo[] nodes = null;
+     StorageType[] storageTypes = null;
 -    int count = dfsClient.getConf().getNumBlockWriteRetry();
++    int count = getNumBlockWriteRetry();
+     boolean success = false;
+     ExtendedBlock oldBlock = block;
+     do {
 -      errorState.reset();
++      errorState.resetInternalError();
+       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,
++  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));
++      LOG.debug("pipeline = " + Arrays.toString(nodes) + ", " + this);
+     }
+ 
+     // 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();
++        errorState.resetInternalError();
+       } catch (IOException ie) {
+         if (!errorState.isRestartingNode()) {
 -          LOG.info("Exception in createBlockOutputStream", ie);
++          LOG.info("Exception in createBlockOutputStream " + this, 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);
++        errorState.setInternalError();
+         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)
++  private 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;
 -          }
 -
 -        }
 -      }
 -    }
++    return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block,
++        stat.getFileId(), favoredNodes);
+   }
+ 
+   /**
+    * 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;
+   }
+ 
++  String[] getStorageIDs() {
++    return storageIDs;
++  }
++
++  BlockConstructionStage getStage() {
++    return stage;
++  }
++
+   /**
+    * return the token of the block
+    *
+    * @return the token of the block
+    */
+   Token<BlockTokenIdentifier> getBlockToken() {
+     return accessToken;
+   }
+ 
++  ErrorState getErrorState() {
++    return errorState;
++  }
++
+   /**
+    * 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(Tracer.getCurrentSpanId());
+       dataQueue.addLast(packet);
+       lastQueuedSeqno = packet.getSeqno();
+       if (LOG.isDebugEnabled()) {
 -        LOG.debug("Queued packet " + packet.getSeqno());
++        LOG.debug("Queued " + packet + ", " + this);
+       }
+       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());
++    return block == null? "block==null": "" + block.getLocalBlock();
+   }
+ }


[58/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

Change-Id: I53ec1c426dc988d6c4a2c87b00caef49c4057010


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

Branch: refs/heads/HDFS-7285
Commit: 6c17d315287020368689fa078a40a1eaedf89d5b
Parents: 8fd5520 854d25b
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Wed Sep 30 08:39:13 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Wed Sep 30 08:39:13 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   5 +
 .../org/apache/hadoop/fs/FSOutputSummer.java    |   3 -
 .../main/java/org/apache/hadoop/util/GSet.java  |  14 ++
 .../org/apache/hadoop/util/GSetByHashMap.java   |   6 +
 .../org/apache/hadoop/util/LightWeightGSet.java |  82 ++++--
 .../hadoop/util/LightWeightResizableGSet.java   | 129 ++++++++++
 .../hadoop-common/src/site/markdown/Tracing.md  |  15 +-
 .../apache/hadoop/fs/shell/TestTextCommand.java |  15 +-
 .../java/org/apache/hadoop/util/TestGSet.java   |  69 ++++-
 .../hadoop/util/TestLightWeightCache.java       |   6 +
 .../util/TestLightWeightResizableGSet.java      | 252 +++++++++++++++++++
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 114 +++------
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  77 ++----
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  20 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  36 +--
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  85 +++----
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  36 +--
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  40 +--
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   4 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  25 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |   2 -
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  18 +-
 .../protocol/datatransfer/PacketReceiver.java   |   8 +-
 .../hdfs/protocol/datatransfer/Sender.java      |   6 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java |   8 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  10 +-
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  45 +---
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 101 ++------
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  33 +--
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  14 +-
 .../hadoop/hdfs/util/ByteArrayManager.java      |  63 +----
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |   8 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |   4 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  28 +--
 .../services/org.apache.hadoop.fs.FileSystem    |   1 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  21 ++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   9 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  37 +--
 .../hadoop/hdfs/server/datanode/DataNode.java   |  11 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |   8 -
 .../hdfs/server/datanode/ReplicaInfo.java       |  27 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |   7 +-
 .../datanode/fsdataset/impl/ReplicaMap.java     |  38 +--
 .../server/datanode/web/DatanodeHttpServer.java |  15 ++
 .../hdfs/server/namenode/NameNodeRpcServer.java |   2 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   5 +-
 .../services/org.apache.hadoop.fs.FileSystem    |  16 --
 .../src/main/resources/hdfs-default.xml         |   8 -
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     | 144 ++++++++---
 .../blockmanagement/TestBlockTokenWithDFS.java  |   2 +
 .../datanode/TestDataNodeVolumeFailure.java     |   2 +
 .../apache/hadoop/tracing/TestTraceAdmin.java   |   8 +-
 hadoop-yarn-project/CHANGES.txt                 |   6 +
 .../localizer/LocalResourcesTracker.java        |   3 +-
 .../localizer/LocalResourcesTrackerImpl.java    |  24 +-
 .../localizer/ResourceLocalizationService.java  |   5 +-
 .../TestLocalResourcesTrackerImpl.java          |  65 ++++-
 .../TestResourceLocalizationService.java        |  16 +-
 .../scheduler/fair/QueueManager.java            |  80 ++++--
 60 files changed, 1157 insertions(+), 794 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index be346a4,8f3df81..183602a
--- 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
@@@ -3103,48 -3049,9 +3091,46 @@@ public class DFSClient implements java.
        }
      });
      HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
-     if (LOG.isDebugEnabled()) {
-       LOG.debug("Using hedged reads; pool threads=" + num);
-     }
+     LOG.debug("Using hedged reads; pool threads={}", num);
    }
  
 +  /**
 +   * Create thread pool for parallel reading in striped layout,
 +   * STRIPED_READ_THREAD_POOL, if it does not already exist.
 +   * @param num Number of threads for striped reads thread pool.
 +   */
 +  private void initThreadsNumForStripedReads(int num) {
 +    assert num > 0;
 +    if (STRIPED_READ_THREAD_POOL != null) {
 +      return;
 +    }
 +    synchronized (DFSClient.class) {
 +      if (STRIPED_READ_THREAD_POOL == null) {
 +        STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
 +            TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
 +            new Daemon.DaemonFactory() {
 +          private final AtomicInteger threadIndex = new AtomicInteger(0);
 +
 +          @Override
 +          public Thread newThread(Runnable r) {
 +            Thread t = super.newThread(r);
 +            t.setName("stripedRead-" + threadIndex.getAndIncrement());
 +            return t;
 +          }
 +        }, new ThreadPoolExecutor.CallerRunsPolicy() {
 +          @Override
 +          public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
 +            LOG.info("Execution for striped reading rejected, "
 +                + "Executing in current thread");
 +            // will run in the current thread
 +            super.rejectedExecution(runnable, e);
 +          }
 +        });
 +        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
 +      }
 +    }
 +  }
 +
    ThreadPoolExecutor getHedgedReadsThreadPool() {
      return HEDGED_READ_THREAD_POOL;
    }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 78eaa6c,47c4b7e..6039177
--- 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
@@@ -191,11 -190,11 +191,11 @@@ public class DFSOutputStream extends FS
      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);
+     if (progress != null) {
+       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
+           +"{}", src);
      }
 -    
 +
      this.bytesPerChecksum = checksum.getBytesPerChecksum();
      if (bytesPerChecksum <= 0) {
        throw new HadoopIllegalArgumentException(
@@@ -410,14 -394,10 +407,10 @@@
      if (currentPacket == null) {
        currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
            .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
-       if (LOG.isDebugEnabled()) {
-         LOG.debug("WriteChunk allocating new packet seqno=" +
-             currentPacket.getSeqno() +
-             ", src=" + src +
-             ", packetSize=" + packetSize +
-             ", chunksPerPacket=" + chunksPerPacket +
-             ", bytesCurBlock=" + getStreamer().getBytesCurBlock() + ", " + this);
-       }
 -      DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno={},"
++      DFSClient.LOG.debug("WriteChunk allocating new packet seqno={},"
+               + " src={}, packetSize={}, chunksPerPacket={}, bytesCurBlock={}",
+           currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
 -          getStreamer().getBytesCurBlock());
++          getStreamer().getBytesCurBlock() + ", " + this);
      }
  
      currentPacket.writeChecksum(checksum, ckoff, cklen);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index af7a61e,4ea1f41..683d98d
--- 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
@@@ -507,10 -479,8 +503,8 @@@ class DataStreamer extends Daemon 
      stage = BlockConstructionStage.DATA_STREAMING;
    }
  
 -  private void endBlock() {
 -    LOG.debug("Closing old block {}", block);
 +  protected void endBlock() {
-     if(LOG.isDebugEnabled()) {
-       LOG.debug("Closing old block " + block);
-     }
++    LOG.debug("Closing old block " + block);
      this.setName("DataStreamer for file " + src);
      closeResponder();
      closeStream();
@@@ -590,10 -560,8 +584,11 @@@
          }
  
          // get new block from namenode.
 +        if (LOG.isDebugEnabled()) {
 +          LOG.debug("stage=" + stage + ", " + this);
 +        }
          if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+           LOG.debug("Allocating new block");
            setPipeline(nextBlockOutputStream());
            initDataStreaming();
          } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
@@@ -645,9 -617,7 +641,7 @@@
            }
          }
  
-         if (LOG.isDebugEnabled()) {
-           LOG.debug(this + " sending " + one);
-         }
 -        LOG.debug("DataStreamer block {} sending packet {}", block, one);
++        LOG.debug(this + " sending " + one);
  
          // write out data to remote datanode
          TraceScope writeScope = dfsClient.getTracer().
@@@ -1785,9 -1768,7 +1773,7 @@@
        packet.addTraceParent(Tracer.getCurrentSpanId());
        dataQueue.addLast(packet);
        lastQueuedSeqno = packet.getSeqno();
-       if (LOG.isDebugEnabled()) {
-         LOG.debug("Queued " + packet + ", " + this);
-       }
 -      LOG.debug("Queued packet {}", packet.getSeqno());
++      LOG.debug("Queued " + packet + ", " + this);
        dataQueue.notifyAll();
      }
    }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------


[50/58] [abbrv] hadoop git commit: HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai.

Posted by zh...@apache.org.
HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai.


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

Branch: refs/heads/HDFS-7285
Commit: d5a9a3daa0224249221ffa7b8bd5751ab2feca56
Parents: 6f335e4
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 29 17:48:29 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 29 17:48:29 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 114 ++++++-------------
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  77 ++++---------
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  20 +---
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  36 ++----
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  85 +++++---------
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  36 +++---
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  47 +++-----
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   4 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  25 ++--
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  18 +--
 .../protocol/datatransfer/PacketReceiver.java   |   8 +-
 .../hdfs/protocol/datatransfer/Sender.java      |   6 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java |   8 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  10 +-
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  45 +++-----
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 101 ++++------------
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  33 ++----
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  14 +--
 .../hadoop/hdfs/util/ByteArrayManager.java      |  63 ++--------
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |   8 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |   4 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  28 ++---
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 24 files changed, 230 insertions(+), 573 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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
index 4f37090..f249692 100644
--- 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
@@ -349,17 +349,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       if (clientContext.getUseLegacyBlockReaderLocal()) {
         reader = getLegacyBlockReaderLocal();
         if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new legacy block reader local.");
-          }
+          LOG.trace("{}: returning new legacy block reader local.", this);
           return reader;
         }
       } else {
         reader = getBlockReaderLocal();
         if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new block reader local.");
-          }
+          LOG.trace("{}: returning new block reader local.", this);
           return reader;
         }
       }
@@ -367,10 +363,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     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());
-        }
+        LOG.trace("{}: returning new remote block reader using UNIX domain "
+            + "socket on {}", this, pathInfo.getPath());
         return reader;
       }
     }
@@ -405,10 +399,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             setVisibleLength(visibleLength).
             build();
         if (accessor == null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": No ReplicaAccessor created by " +
-                cls.getName());
-          }
+          LOG.trace("{}: No ReplicaAccessor created by {}",
+              this, cls.getName());
         } else {
           return new ExternalBlockReader(accessor, visibleLength, startOffset);
         }
@@ -427,14 +419,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * first introduced in HDFS-2246.
    */
   private BlockReader getLegacyBlockReaderLocal() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
-    }
+    LOG.trace("{}: trying to construct BlockReaderLocalLegacy", this);
     if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
-            "the address " + inetSocketAddress + " is not local");
-      }
+      LOG.trace("{}: can't construct BlockReaderLocalLegacy because the address"
+          + "{} is not local", this, inetSocketAddress);
       return null;
     }
     if (clientContext.getDisableLegacyBlockReaderLocal()) {
@@ -470,10 +458,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   }
 
   private BlockReader getBlockReaderLocal() throws InvalidToken {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct a BlockReaderLocal " +
-          "for short-circuit reads.");
-    }
+    LOG.trace("{}: trying to construct a BlockReaderLocal for short-circuit "
+        + " reads.", this);
     if (pathInfo == null) {
       pathInfo = clientContext.getDomainSocketFactory()
           .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
@@ -488,10 +474,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     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());
-      }
+      LOG.trace("{}: got InvalidToken exception while trying to construct "
+          + "BlockReaderLocal via {}", this, pathInfo.getPath());
       throw exc;
     }
     if (info.getReplica() == null) {
@@ -527,9 +511,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
       if (info != null) return info;
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
-    }
+    LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
     BlockReaderPeer curPeer;
     while (true) {
       curPeer = nextDomainPeer();
@@ -544,10 +526,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             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...");
-          }
+          LOG.trace("{}: allocShmSlot used up our previous socket {}.  "
+              + "Allocating a new one...", this, peer.getDomainSocket());
           curPeer = nextDomainPeer();
           if (curPeer == null) break;
           peer = (DomainPeer)curPeer.peer;
@@ -562,9 +542,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         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);
-          }
+          LOG.debug("{}: closing stale domain peer {}", this, peer, e);
           IOUtilsClient.cleanup(LOG, peer);
         } else {
           // Handle an I/O error we got when using a newly created socket.
@@ -617,7 +595,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
         if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
-          LOG.trace("Sending receipt verification byte for slot " + slot);
+          LOG.trace("Sending receipt verification byte for slot {}", slot);
           sock.getOutputStream().write(0);
         }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
@@ -650,9 +628,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       String msg = "access control error while " +
           "attempting to set up short-circuit access to " +
           fileName + resp.getMessage();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(this + ":" + msg);
-      }
+      LOG.debug("{}:{}", this, msg);
       return new ShortCircuitReplicaInfo(new InvalidToken(msg));
     default:
       LOG.warn(this + ": unknown response code " + resp.getStatus() +
@@ -684,10 +660,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
            " 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());
-    }
+    LOG.trace("{}: trying to create a remote block reader from the UNIX domain "
+        + "socket at {}", this, pathInfo.getPath());
 
     while (true) {
       BlockReaderPeer curPeer = nextDomainPeer();
@@ -701,19 +675,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       } 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);
-          }
+          LOG.trace("{}: got security exception while constructing a remote "
+                  + " block reader from the unix domain socket at {}",
+              this, 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);
-          }
+          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
@@ -747,10 +717,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    *             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");
-    }
+    LOG.trace("{}: trying to create a remote block reader from a TCP socket",
+        this);
     BlockReader blockReader = null;
     while (true) {
       BlockReaderPeer curPeer = null;
@@ -763,19 +731,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         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);
-          }
+          LOG.trace("{}: got security exception while constructing a remote "
+              + "block reader from {}", this, 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);
-          }
+          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);
@@ -808,9 +772,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, true);
       if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextDomainPeer: reusing existing peer " + peer);
-        }
+        LOG.trace("nextDomainPeer: reusing existing peer {}", peer);
         return new BlockReaderPeer(peer, true);
       }
     }
@@ -832,24 +794,18 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, false);
       if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextTcpPeer: reusing existing peer " + peer);
-        }
+        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);
-      }
+      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);
-      }
+      LOG.trace("nextTcpPeer: failed to create newConnectedPeer connected to"
+          + "{}", datanode);
       throw e;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index 62e7af6..10dc35c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@ -412,17 +412,10 @@ class BlockReaderLocal implements BlockReader {
   public synchronized int read(ByteBuffer buf) throws IOException {
     boolean canSkipChecksum = createNoChecksumContext();
     try {
-      String traceString = null;
-      if (LOG.isTraceEnabled()) {
-        traceString = new StringBuilder().
-            append("read(").
-            append("buf.remaining=").append(buf.remaining()).
-            append(", block=").append(block).
-            append(", filename=").append(filename).
-            append(", canSkipChecksum=").append(canSkipChecksum).
-            append(")").toString();
-        LOG.info(traceString + ": starting");
-      }
+      String traceFormatStr = "read(buf.remaining={}, block={}, filename={}, "
+          + "canSkipChecksum={})";
+      LOG.trace(traceFormatStr + ": starting",
+          buf.remaining(), block, filename, canSkipChecksum);
       int nRead;
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
@@ -431,14 +424,12 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(buf, canSkipChecksum);
         }
       } catch (IOException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.info(traceString + ": I/O error", e);
-        }
+        LOG.trace(traceFormatStr + ": I/O error",
+            buf.remaining(), block, filename, canSkipChecksum, e);
         throw e;
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.info(traceString + ": returning " + nRead);
-      }
+      LOG.trace(traceFormatStr + ": returning {}",
+          buf.remaining(), block, filename, canSkipChecksum, nRead);
       return nRead;
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
@@ -490,10 +481,8 @@ class BlockReaderLocal implements BlockReader {
     }
     dataBuf.limit(dataBuf.position());
     dataBuf.position(Math.min(dataBuf.position(), slop));
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("loaded " + dataBuf.remaining() + " bytes into bounce " +
-          "buffer from offset " + oldDataPos + " of " + block);
-    }
+    LOG.trace("loaded {} bytes into bounce buffer from offset {} of {}",
+        dataBuf.remaining(), oldDataPos, block);
     return dataBuf.limit() != maxReadaheadLength;
   }
 
@@ -565,18 +554,10 @@ class BlockReaderLocal implements BlockReader {
     boolean canSkipChecksum = createNoChecksumContext();
     int nRead;
     try {
-      String traceString = null;
-      if (LOG.isTraceEnabled()) {
-        traceString = new StringBuilder().
-            append("read(arr.length=").append(arr.length).
-            append(", off=").append(off).
-            append(", len=").append(len).
-            append(", filename=").append(filename).
-            append(", block=").append(block).
-            append(", canSkipChecksum=").append(canSkipChecksum).
-            append(")").toString();
-        LOG.trace(traceString + ": starting");
-      }
+      final String traceFormatStr = "read(arr.length={}, off={}, len={}, "
+          + "filename={}, block={}, canSkipChecksum={})";
+      LOG.trace(traceFormatStr + ": starting",
+          arr.length, off, len, filename, block, canSkipChecksum);
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
           nRead = readWithoutBounceBuffer(arr, off, len);
@@ -584,14 +565,12 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
         }
       } catch (IOException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(traceString + ": I/O error", e);
-        }
+        LOG.trace(traceFormatStr + ": I/O error",
+            arr.length, off, len, filename, block, canSkipChecksum, e);
         throw e;
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(traceString + ": returning " + nRead);
-      }
+      LOG.trace(traceFormatStr + ": returning {}",
+          arr.length, off, len, filename, block, canSkipChecksum, nRead);
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
     }
@@ -634,11 +613,9 @@ class BlockReaderLocal implements BlockReader {
       dataBuf.position(dataBuf.position() + discardedFromBuf);
       remaining -= discardedFromBuf;
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("skip(n=" + n + ", block=" + block + ", filename=" + 
-        filename + "): discarded " + discardedFromBuf + " bytes from " +
-        "dataBuf and advanced dataPos by " + remaining);
-    }
+    LOG.trace("skip(n={}, block={}, filename={}): discarded {} bytes from "
+            + "dataBuf and advanced dataPos by {}",
+        n, block, filename, discardedFromBuf, remaining);
     dataPos += remaining;
     return n;
   }
@@ -653,9 +630,7 @@ class BlockReaderLocal implements BlockReader {
   public synchronized void close() throws IOException {
     if (closed) return;
     closed = true;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("close(filename=" + filename + ", block=" + block + ")");
-    }
+    LOG.trace("close(filename={}, block={})", filename, block);
     replica.unref();
     freeDataBufIfExists();
     freeChecksumBufIfExists();
@@ -705,11 +680,9 @@ class BlockReaderLocal implements BlockReader {
         (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
     if (anchor) {
       if (!createNoChecksumContext()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("can't get an mmap for " + block + " of " + filename + 
-              " since SKIP_CHECKSUMS was not given, " +
-              "we aren't skipping checksums, and the block is not mlocked.");
-        }
+        LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
+            + "given, we aren't skipping checksums, and the block is not "
+            + "mlocked.", block, filename);
         return null;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 5235287..4a1828e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -221,11 +221,9 @@ class BlockReaderLocalLegacy implements BlockReader {
       File blkfile = new File(pathinfo.getBlockPath());
       dataIn = new FileInputStream(blkfile);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("New BlockReaderLocalLegacy for file " + blkfile + " of size "
-            + blkfile.length() + " startOffset " + startOffset + " length "
-            + length + " short circuit checksum " + !skipChecksumCheck);
-      }
+      LOG.debug("New BlockReaderLocalLegacy for file {} of size {} startOffset "
+              + "{} length {} short circuit checksum {}",
+          blkfile, blkfile.length(), startOffset, length, !skipChecksumCheck);
 
       if (!skipChecksumCheck) {
         // get the metadata file
@@ -292,9 +290,7 @@ class BlockReaderLocalLegacy implements BlockReader {
       // channel for the DataNode to notify the client that the path has been
       // invalidated.  Therefore, our only option is to skip caching.
       if (pathinfo != null && !storageType.isTransient()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Cached location of block " + blk + " as " + pathinfo);
-        }
+        LOG.debug("Cached location of block {} as {}", blk, pathinfo);
         localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
       }
     } catch (IOException e) {
@@ -603,9 +599,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized int read(byte[] buf, int off, int len) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("read off " + off + " len " + len);
-    }
+    LOG.trace("read off {} len {}", off, len);
     if (!verifyChecksum) {
       return dataIn.read(buf, off, len);
     }
@@ -624,9 +618,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized long skip(long n) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("skip " + n);
-    }
+    LOG.debug("skip {}", n);
     if (n <= 0) {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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
index 92d117c..8f3df81 100644
--- 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
@@ -419,9 +419,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     final int idx = r.nextInt(localInterfaceAddrs.length);
     final SocketAddress addr = localInterfaceAddrs[idx];
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using local interface " + addr);
-    }
+    LOG.debug("Using local interface {}", addr);
     return addr;
   }
 
@@ -1216,9 +1214,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                              InetSocketAddress[] favoredNodes) throws IOException {
     checkOpen();
     final FsPermission masked = applyUMask(permission);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + masked);
-    }
+    LOG.debug("{}: masked={}", src, masked);
     final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
         src, masked, flag, createParent, replication, blockSize, progress,
         buffersize, dfsClientConf.createChecksum(checksumOpt),
@@ -1815,10 +1811,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
               smallBufferSize));
           in = new DataInputStream(pair.in);
 
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("write to " + datanodes[j] + ": "
-                + Op.BLOCK_CHECKSUM + ", block=" + block);
-          }
+          LOG.debug("write to {}: {}, block={}",
+              datanodes[j], Op.BLOCK_CHECKSUM, block);
           // get block MD5
           new Sender(out).blockChecksum(block, lb.getBlockToken());
 
@@ -1882,12 +1876,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           }
         } 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.");
-            }
+            LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
+                  + "for file {} for block {} from datanode {}. Will retry the "
+                  + "block once.",
+                src, block, datanodes[j]);
             lastRetriedIndex = i;
             done = true; // actually it's not done; but we'll retry
             i--; // repeat at i-th block
@@ -1941,9 +1933,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try {
       sock = socketFactory.createSocket();
       String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Connecting to datanode " + dnAddr);
-      }
+      LOG.debug("Connecting to datanode {}", dnAddr);
       NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
       sock.setSoTimeout(timeout);
   
@@ -2563,9 +2553,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       absPermission = applyUMask(null);
     } 
 
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + absPermission);
-    }
+    LOG.debug("{}: masked={}", src, absPermission);
     TraceScope scope = tracer.newScope("mkdir");
     try {
       return namenode.mkdirs(src, absPermission, createParent);
@@ -3061,9 +3049,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       }
     });
     HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using hedged reads; pool threads=" + num);
-    }
+    LOG.debug("Using hedged reads; pool threads={}", num);
   }
 
   ThreadPoolExecutor getHedgedReadsThreadPool() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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
index 7101753..81e8c27 100644
--- 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
@@ -315,9 +315,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if (locatedBlocks == null || refresh) {
       newInfo = dfsClient.getLocatedBlocks(src, 0);
     }
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("newInfo = " + newInfo);
-    }
+    DFSClient.LOG.debug("newInfo = {}", newInfo);
     if (newInfo == null) {
       throw new IOException("Cannot open filename " + src);
     }
@@ -383,10 +381,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           replicaNotFoundCount--;
         }
         
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
-              + datanode + " for block " + locatedblock.getBlock(), ioe);
-        }
+        DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
+              + " for block {}", datanode, locatedblock.getBlock(), ioe);
       } finally {
         if (cdp != null) {
           RPC.stopProxy(cdp);
@@ -1067,9 +1063,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     final String dnAddr =
         chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
-    }
+    DFSClient.LOG.debug("Connecting to datanode {}", dnAddr);
     InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
     return new DNAddrPair(chosenNode, targetAddr, storageType);
   }
@@ -1309,11 +1303,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             future.get();
             return;
           }
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
-                + "ms to read from " + chosenNode.info
-                + "; spawning hedged read");
-          }
+          DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
+              + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
           // Ignore this node on next go around.
           ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
@@ -1340,10 +1331,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
               .submit(getFromDataNodeCallable);
           futures.add(oneMoreRequest);
         } catch (IOException ioe) {
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Failed getting node for hedged read: "
-                + ioe.getMessage());
-          }
+          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.
@@ -1599,11 +1588,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             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);
-          }
+          DFSClient.LOG.debug("Exception while seek to {} from {} of {} from "
+              + "{}", targetPos, getCurrentBlock(), src, currentNode, e);
         }
       }
     }
@@ -1819,20 +1805,16 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     } 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);
-        }
+        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {}"
+                + " of {}; {} bytes left in block. blockPos={}; curPos={};"
+                + "curEnd={}",
+            curPos, src, length63, blockPos, curPos, 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);
-      }
+      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid going "
+              + "more than one byte past the end of the block.  blockPos={}; "
+              +" curPos={}; curEnd={}",
+          maxLength, length63, blockPos, curPos, curEnd);
     }
     // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
     int length;
@@ -1846,28 +1828,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         // 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);
-        }
+        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {} "
+            + " of {}; 31-bit MappedByteBuffer limit exceeded.  blockPos={}, "
+            + "curEnd={}", curPos, src, blockPos, 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);
-      }
+      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid 31-bit "
+          + "limit.  blockPos={}; curPos={}; curEnd={}",
+          maxLength, length, blockPos, curPos, 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.");
-      }
+      DFSClient.LOG.debug("unable to perform a zero-copy read from offset {} of"
+          + " {}; BlockReader#getClientMmap returned null.", curPos, src);
       return null;
     }
     boolean success = false;
@@ -1881,11 +1855,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       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);
-      }
+      DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
+          + "zero-copy read path.  blockEnd = {}", length, curPos, blockEnd);
       success = true;
     } finally {
       if (!success) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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
index 7a40d73..47c4b7e 100755
--- 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
@@ -190,9 +190,9 @@ public class DFSOutputStream extends FSOutputSummer
     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);
+    if (progress != null) {
+      DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
+          +"{}", src);
     }
     
     this.bytesPerChecksum = checksum.getBytesPerChecksum();
@@ -365,12 +365,9 @@ public class DFSOutputStream extends FSOutputSummer
     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);
-    }
+    DFSClient.LOG.debug("computePacketChunkSize: src={}, chunkSize={}, "
+            + "chunksPerPacket={}, packetSize={}",
+        src, chunkSize, chunksPerPacket, packetSize);
   }
 
   protected TraceScope createWriteTraceScope() {
@@ -397,14 +394,10 @@ public class DFSOutputStream extends FSOutputSummer
     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());
-      }
+      DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno={},"
+              + " src={}, packetSize={}, chunksPerPacket={}, bytesCurBlock={}",
+          currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
+          getStreamer().getBytesCurBlock());
     }
 
     currentPacket.writeChecksum(checksum, ckoff, cklen);
@@ -558,12 +551,9 @@ public class DFSOutputStream extends FSOutputSummer
         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);
-        }
+        DFSClient.LOG.debug("DFSClient flush():  bytesCurBlock={}, "
+                + "lastFlushOffset={}, createNewBlock={}",
+            getStreamer().getBytesCurBlock(), lastFlushOffset, endBlock);
         // Flush only if we haven't already flushed till this offset.
         if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
           assert getStreamer().getBytesCurBlock() > lastFlushOffset;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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 e275afb..71ce7cc 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
@@ -462,19 +462,13 @@ public class DFSUtilClient {
     InetAddress addr = targetAddr.getAddress();
     Boolean cached = localAddrMap.get(addr.getHostAddress());
     if (cached != null) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Address " + targetAddr +
-            (cached ? " is local" : " is not local"));
-      }
+      LOG.trace("Address {} is {} local", targetAddr, (cached ? "" : "not"));
       return cached;
     }
 
     boolean local = NetUtils.isLocalAddress(addr);
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Address " + targetAddr +
-          (local ? " is local" : " is not local"));
-    }
+    LOG.trace("Address {} is {} local", targetAddr, (local ? "" : "not"));
     localAddrMap.put(addr.getHostAddress(), local);
     return local;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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
index d1d8d37..4ea1f41 100644
--- 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
@@ -132,18 +132,14 @@ class DataStreamer extends Daemon {
       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);
-    }
+    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());
-    }
+    LOG.debug("Send buf size {}", sock.getSendBufferSize());
     return sock;
   }
 
@@ -484,9 +480,7 @@ class DataStreamer extends Daemon {
   }
 
   private void endBlock() {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Closing old block " + block);
-    }
+    LOG.debug("Closing old block {}", block);
     this.setName("DataStreamer for file " + src);
     closeResponder();
     closeStream();
@@ -567,15 +561,11 @@ class DataStreamer extends Daemon {
 
         // get new block from namenode.
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Allocating new block");
-          }
+          LOG.debug("Allocating new block");
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Append to block " + block);
-          }
+          LOG.debug("Append to block {}", block);
           setupPipelineForAppendOrRecovery();
           if (streamerClosed) {
             continue;
@@ -627,10 +617,7 @@ class DataStreamer extends Daemon {
           }
         }
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("DataStreamer block " + block +
-              " sending packet " + one);
-        }
+        LOG.debug("DataStreamer block {} sending packet {}", block, one);
 
         // write out data to remote datanode
         TraceScope writeScope = dfsClient.getTracer().
@@ -741,9 +728,7 @@ class DataStreamer extends Daemon {
     TraceScope scope = dfsClient.getTracer().
         newScope("waitForAckedSeqno");
     try {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Waiting for ack for: " + seqno);
-      }
+      LOG.debug("Waiting for ack for: {}", seqno);
       long begin = Time.monotonicNow();
       try {
         synchronized (dataQueue) {
@@ -955,8 +940,8 @@ class DataStreamer extends Daemon {
             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);
+          } else {
+            LOG.debug("DFSClient {}", ack);
           }
 
           long seqno = ack.getSeqno();
@@ -1176,9 +1161,7 @@ class DataStreamer extends Daemon {
   }
 
   private void addDatanode2ExistingPipeline() throws IOException {
-    if (DataTransferProtocol.LOG.isDebugEnabled()) {
-      DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
-    }
+    DataTransferProtocol.LOG.debug("lastAckedSeqno = {}", lastAckedSeqno);
       /*
        * Is data transfer necessary?  We have the following cases.
        *
@@ -1645,10 +1628,8 @@ class DataStreamer extends Daemon {
           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] + ").");
-        }
+        LOG.debug("{} was chosen by name node (favored={}).",
+            nodes[i].getXferAddrWithHostname(), pinnings[i]);
       }
       if (shouldLog && !favoredSet.isEmpty()) {
         // There is one or more favored nodes that were not allocated.
@@ -1787,9 +1768,7 @@ class DataStreamer extends Daemon {
       packet.addTraceParent(Tracer.getCurrentSpanId());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Queued packet " + packet.getSeqno());
-      }
+      LOG.debug("Queued packet {}", packet.getSeqno());
       dataQueue.notifyAll();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index 15a5bee..017be9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -250,9 +250,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       PacketHeader header = new PacketHeader();
       header.readFields(in);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("DFSClient readChunk got header " + header);
-      }
+      LOG.debug("DFSClient readChunk got header {}", header);
 
       // Sanity check the lengths
       if (!header.sanityCheck(lastSeqNo)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 7a7932d..ca31e67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -135,14 +135,9 @@ public class RemoteBlockReader2  implements BlockReader {
   @Override
   public synchronized int read(byte[] buf, int off, int len) 
                                throws IOException {
-
-    UUID randomId = null;
-    if (LOG.isTraceEnabled()) {
-      randomId = UUID.randomUUID();
-      LOG.trace(String.format("Starting read #%s file %s from datanode %s",
-        randomId.toString(), this.filename,
-        this.datanodeID.getHostName()));
-    }
+    UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
+    LOG.trace("Starting read #{} file {} from datanode {}",
+        randomId, filename, datanodeID.getHostName());
 
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
       TraceScope scope = tracer.newScope(
@@ -154,9 +149,7 @@ public class RemoteBlockReader2  implements BlockReader {
       }
     }
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("Finishing read #" + randomId));
-    }
+    LOG.trace("Finishing read #{}", randomId);
 
     if (curDataSlice.remaining() == 0) {
       // we're at EOF now
@@ -203,9 +196,7 @@ public class RemoteBlockReader2  implements BlockReader {
     curDataSlice = packetReceiver.getDataSlice();
     assert curDataSlice.capacity() == curHeader.getDataLen();
     
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("DFSClient readNextPacket got header " + curHeader);
-    }
+    LOG.trace("DFSClient readNextPacket got header {}", curHeader);
 
     // Sanity check the lengths
     if (!curHeader.sanityCheck(lastSeqNo)) {
@@ -276,10 +267,8 @@ public class RemoteBlockReader2  implements BlockReader {
   }
 
   private void readTrailingEmptyPacket() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Reading empty packet at end of read");
-    }
-    
+    LOG.trace("Reading empty packet at end of read");
+
     packetReceiver.receiveNextPacket(in);
 
     PacketHeader trailer = packetReceiver.getHeader();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/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
index c3d2cfc..8457d65 100644
--- 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
@@ -308,10 +308,7 @@ public class LeaseRenewer {
               }
               LeaseRenewer.this.run(id);
             } catch(InterruptedException e) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
-                    + " is interrupted.", e);
-              }
+              LOG.debug("LeaseRenewer is interrupted.", e);
             } finally {
               synchronized(LeaseRenewer.this) {
                 Factory.INSTANCE.remove(LeaseRenewer.this);
@@ -399,9 +396,7 @@ public class LeaseRenewer {
     }
 
     if (daemonCopy != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Wait for lease checker to terminate");
-      }
+      LOG.debug("Wait for lease checker to terminate");
       daemonCopy.join();
     }
   }
@@ -424,16 +419,11 @@ public class LeaseRenewer {
       //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);
-          }
+          LOG.debug("Did not renew lease for client {}", c);
           continue;
         }
         previousName = c.getClientName();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Lease renewed for client " + previousName);
-        }
+        LOG.debug("Lease renewed for client {}", previousName);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
index c4093b1..e6709d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
@@ -147,11 +147,9 @@ public class PacketReceiver implements Closeable {
       throw new IOException("Invalid header length " + headerLen);
     }
     
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("readNextPacket: dataPlusChecksumLen = " + dataPlusChecksumLen +
-          " headerLen = " + headerLen);
-    }
-    
+    LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
+        dataPlusChecksumLen, headerLen);
+
     // Sanity check the buffer size so we don't allocate too much memory
     // and OOME.
     int totalLen = payloadLen + headerLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index e856211..d2bc348 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -73,10 +73,8 @@ public class Sender implements DataTransferProtocol {
 
   private static void send(final DataOutputStream out, final Op opcode,
       final Message proto) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Sending DataTransferOp " + proto.getClass().getSimpleName()
-          + ": " + proto);
-    }
+    LOG.trace("Sending DataTransferOp {}: {}",
+        proto.getClass().getSimpleName(), proto);
     op(out, opcode);
     proto.writeDelimitedTo(out);
     out.flush();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
index 256caff..006d304 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
@@ -332,11 +332,9 @@ public final class DataTransferSaslUtil {
   public static IOStreamPair createStreamPair(Configuration conf,
       CipherOption cipherOption, OutputStream out, InputStream in, 
       boolean isServer) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Creating IOStreamPair of CryptoInputStream and " +
-          "CryptoOutputStream.");
-    }
-    CryptoCodec codec = CryptoCodec.getInstance(conf, 
+    LOG.debug("Creating IOStreamPair of CryptoInputStream and "
+        + "CryptoOutputStream.");
+    CryptoCodec codec = CryptoCodec.getInstance(conf,
         cipherOption.getCipherSuite());
     byte[] inKey = cipherOption.getInKey();
     byte[] inIv = cipherOption.getInIv();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index f764275..24e1dd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -130,9 +130,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
-    }
+    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
     rpcProxy = createClientDatanodeProtocolProxy(addr,
         UserGroupInformation.getCurrentUser(), conf,
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
@@ -143,10 +141,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
       boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
-    }
-    
+    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
+
     // Since we're creating a new UserGroupInformation here, we know that no
     // future RPC proxies will be able to re-use the same connection. And
     // usages of this proxy tend to be one-off calls.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
index f70398a..4ffc108 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
@@ -129,18 +129,13 @@ public class DfsClientShmManager implements Closeable {
       ShmId shmId = shm.getShmId();
       Slot slot = shm.allocAndRegisterSlot(blockId);
       if (shm.isFull()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": pulled the last slot " + slot.getSlotIdx() +
-              " out of " + shm);
-        }
+        LOG.trace("{}: pulled the last slot {} out of {}",
+            this, slot.getSlotIdx(), shm);
         DfsClientShm removedShm = notFull.remove(shmId);
         Preconditions.checkState(removedShm == shm);
         full.put(shmId, shm);
       } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": pulled slot " + slot.getSlotIdx() +
-              " out of " + shm);
-        }
+        LOG.trace("{}: pulled slot {} out of {}", this, slot.getSlotIdx(), shm);
       }
       return slot;
     }
@@ -187,9 +182,7 @@ public class DfsClientShmManager implements Closeable {
           DfsClientShm shm = 
               new DfsClientShm(PBHelperClient.convert(resp.getId()),
                   fis[0], this, peer);
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": createNewShm: created " + shm);
-          }
+          LOG.trace("{}: createNewShm: created {}", this, shm);
           return shm;
         } finally {
           try {
@@ -234,15 +227,11 @@ public class DfsClientShmManager implements Closeable {
         String clientName, ExtendedBlockId blockId) throws IOException {
       while (true) {
         if (closed) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": the DfsClientShmManager has been closed.");
-          }
+          LOG.trace("{}: the DfsClientShmManager has been closed.", this);
           return null;
         }
         if (disabled) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": shared memory segment access is disabled.");
-          }
+          LOG.trace("{}: shared memory segment access is disabled.", this);
           return null;
         }
         // Try to use an existing slot.
@@ -253,9 +242,7 @@ public class DfsClientShmManager implements Closeable {
         // There are no free slots.  If someone is loading more slots, wait
         // for that to finish.
         if (loading) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": waiting for loading to finish...");
-          }
+          LOG.trace("{}: waiting for loading to finish...", this);
           finishedLoading.awaitUninterruptibly();
         } else {
           // Otherwise, load the slot ourselves.
@@ -282,11 +269,9 @@ public class DfsClientShmManager implements Closeable {
             // fired and marked the shm as disconnected.  In this case, we
             // obviously don't want to add the SharedMemorySegment to our list
             // of valid not-full segments.
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(this + ": the UNIX domain socket associated with " +
-                  "this short-circuit memory closed before we could make " +
-                  "use of the shm.");
-            }
+            LOG.debug("{}: the UNIX domain socket associated with this "
+                + "short-circuit memory closed before we could make use of "
+                + "the shm.", this);
           } else {
             notFull.put(shm.getShmId(), shm);
           }
@@ -309,9 +294,7 @@ public class DfsClientShmManager implements Closeable {
         Preconditions.checkState(!full.containsKey(shm.getShmId()));
         Preconditions.checkState(!notFull.containsKey(shm.getShmId()));
         if (shm.isEmpty()) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": freeing empty stale " + shm);
-          }
+          LOG.trace("{}: freeing empty stale {}", this, shm);
           shm.free();
         }
       } else {
@@ -336,10 +319,8 @@ public class DfsClientShmManager implements Closeable {
           // lowest ID, but it could still occur.  In most workloads,
           // fragmentation should not be a major concern, since it doesn't impact
           // peak file descriptor usage or the speed of allocation.
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": shutting down UNIX domain socket for " +
-                "empty " + shm);
-          }
+          LOG.trace("{}: shutting down UNIX domain socket for empty {}",
+              this, shm);
           shutdown(shm);
         } else {
           notFull.put(shmId, shm);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
index 52c1a6e..07f5064 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
@@ -103,9 +103,7 @@ public class ShortCircuitCache implements Closeable {
         if (ShortCircuitCache.this.closed) return;
         long curMs = Time.monotonicNow();
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(this + ": cache cleaner running at " + curMs);
-        }
+        LOG.debug("{}: cache cleaner running at {}", this, curMs);
 
         int numDemoted = demoteOldEvictableMmaped(curMs);
         int numPurged = 0;
@@ -127,11 +125,9 @@ public class ShortCircuitCache implements Closeable {
           numPurged++;
         }
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(this + ": finishing cache cleaner run started at " +
-            curMs + ".  Demoted " + numDemoted + " mmapped replicas; " +
-            "purged " + numPurged + " replicas.");
-        }
+        LOG.debug("{}: finishing cache cleaner run started at {}. Demoted {} "
+            + "mmapped replicas; purged {} replicas.",
+            this, curMs, numDemoted, numPurged);
       } finally {
         ShortCircuitCache.this.lock.unlock();
       }
@@ -186,9 +182,7 @@ public class ShortCircuitCache implements Closeable {
 
     @Override
     public void run() {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(ShortCircuitCache.this + ": about to release " + slot);
-      }
+      LOG.trace("{}: about to release {}", ShortCircuitCache.this, slot);
       final DfsClientShm shm = (DfsClientShm)slot.getShm();
       final DomainSocket shmSock = shm.getPeer().getDomainSocket();
       final String path = shmSock.getPath();
@@ -205,9 +199,7 @@ public class ShortCircuitCache implements Closeable {
           String error = resp.hasError() ? resp.getError() : "(unknown)";
           throw new IOException(resp.getStatus().toString() + ": " + error);
         }
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(ShortCircuitCache.this + ": released " + slot);
-        }
+        LOG.trace("{}: released {}", this, slot);
         success = true;
       } catch (IOException e) {
         LOG.error(ShortCircuitCache.this + ": failed to release " +
@@ -433,9 +425,7 @@ public class ShortCircuitCache implements Closeable {
           purgeReason = "purging replica because it is stale.";
         }
         if (purgeReason != null) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(this + ": " + purgeReason);
-          }
+          LOG.debug("{}: {}", this, purgeReason);
           purge(replica);
         }
       }
@@ -677,10 +667,8 @@ public class ShortCircuitCache implements Closeable {
       ShortCircuitReplicaInfo info = null;
       do {
         if (closed) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": can't fetchOrCreate " + key +
-                " because the cache is closed.");
-          }
+          LOG.trace("{}: can't fethchOrCreate {} because the cache is closed.",
+              this, key);
           return null;
         }
         Waitable<ShortCircuitReplicaInfo> waitable = replicaInfoMap.get(key);
@@ -688,9 +676,7 @@ public class ShortCircuitCache implements Closeable {
           try {
             info = fetch(key, waitable);
           } catch (RetriableException e) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(this + ": retrying " + e.getMessage());
-            }
+            LOG.debug("{}: retrying {}", this, e.getMessage());
             continue;
           }
         }
@@ -721,9 +707,7 @@ public class ShortCircuitCache implements Closeable {
     // ShortCircuitReplica.  So we simply wait for it to complete.
     ShortCircuitReplicaInfo info;
     try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": found waitable for " + key);
-      }
+      LOG.trace("{}: found waitable for {}", this, key);
       info = waitable.await();
     } catch (InterruptedException e) {
       LOG.info(this + ": interrupted while waiting for " + key);
@@ -765,9 +749,7 @@ public class ShortCircuitCache implements Closeable {
     // Handle loading a new replica.
     ShortCircuitReplicaInfo info = null;
     try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": loading " + key);
-      }
+      LOG.trace("{}: loading {}", this, key);
       info = creator.createShortCircuitReplicaInfo();
     } catch (RuntimeException e) {
       LOG.warn(this + ": failed to load " + key, e);
@@ -777,9 +759,7 @@ public class ShortCircuitCache implements Closeable {
     try {
       if (info.getReplica() != null) {
         // On success, make sure the cache cleaner thread is running.
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": successfully loaded " + info.getReplica());
-        }
+        LOG.trace("{}: successfully loaded {}", this, info.getReplica());
         startCacheCleanerThreadIfNeeded();
         // Note: new ShortCircuitReplicas start with a refCount of 2,
         // indicating that both this cache and whoever requested the 
@@ -811,10 +791,8 @@ public class ShortCircuitCache implements Closeable {
           cleanerExecutor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
               TimeUnit.MILLISECONDS);
       cacheCleaner.setFuture(future);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(this + ": starting cache cleaner thread which will run " +
-          "every " + rateMs + " ms");
-      }
+      LOG.debug("{}: starting cache cleaner thread which will run every {} ms",
+          this, rateMs);
     }
   }
 
@@ -832,17 +810,12 @@ public class ShortCircuitCache implements Closeable {
           long lastAttemptTimeMs = (Long)replica.mmapData;
           long delta = Time.monotonicNow() - lastAttemptTimeMs;
           if (delta < mmapRetryTimeoutMs) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace(this + ": can't create client mmap for " +
-                  replica + " because we failed to " +
-                  "create one just " + delta + "ms ago.");
-            }
+            LOG.trace("{}: can't create client mmap for {} because we failed to"
+                + " create one just {}ms ago.", this, replica, delta);
             return null;
           }
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": retrying client mmap for " + replica +
-                ", " + delta + " ms after the previous failure.");
-          }
+          LOG.trace("{}: retrying client mmap for {}, {} ms after the previous "
+              + "failure.", this, replica, delta);
         } else if (replica.mmapData instanceof Condition) {
           Condition cond = (Condition)replica.mmapData;
           cond.awaitUninterruptibly();
@@ -965,38 +938,10 @@ public class ShortCircuitCache implements Closeable {
           }
         }
       }
-      if (LOG.isDebugEnabled()) {
-        StringBuilder builder = new StringBuilder();
-        builder.append("visiting ").append(visitor.getClass().getName()).
-            append("with outstandingMmapCount=").append(outstandingMmapCount).
-            append(", replicas=");
-        String prefix = "";
-        for (Entry<ExtendedBlockId, ShortCircuitReplica> entry : replicas.entrySet()) {
-          builder.append(prefix).append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", failedLoads=");
-        for (Entry<ExtendedBlockId, InvalidToken> entry : failedLoads.entrySet()) {
-          builder.append(prefix).append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", evictable=");
-        for (Entry<Long, ShortCircuitReplica> entry : evictable.entrySet()) {
-          builder.append(prefix).append(entry.getKey()).
-              append(":").append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", evictableMmapped=");
-        for (Entry<Long, ShortCircuitReplica> entry : evictableMmapped.entrySet()) {
-          builder.append(prefix).append(entry.getKey()).
-              append(":").append(entry.getValue());
-          prefix = ",";
-        }
-        LOG.debug(builder.toString());
-      }
+      LOG.debug("visiting {} with outstandingMmapCount={}, replicas={}, "
+          + "failedLoads={}, evictable={}, evictableMmapped={}",
+          visitor.getClass().getName(), outstandingMmapCount, replicas,
+          failedLoads, evictable, evictableMmapped);
       visitor.visit(outstandingMmapCount, replicas, failedLoads,
             evictable, evictableMmapped);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
index 37566e2..38cf22b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
@@ -154,25 +154,19 @@ public class ShortCircuitReplica {
       // Check staleness by looking at the shared memory area we use to
       // communicate with the DataNode.
       boolean stale = !slot.isValid();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": checked shared memory segment.  isStale=" + stale);
-      }
+      LOG.trace("{}: checked shared memory segment.  isStale={}", this, stale);
       return stale;
     } else {
       // Fall back to old, time-based staleness method.
       long deltaMs = Time.monotonicNow() - creationTimeMs;
       long staleThresholdMs = cache.getStaleThresholdMs();
       if (deltaMs > staleThresholdMs) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + " is stale because it's " + deltaMs +
-              " ms old, and staleThresholdMs = " + staleThresholdMs);
-        }
+        LOG.trace("{} is stale because it's {} ms old and staleThreadholdMS={}",
+            this, deltaMs, staleThresholdMs);
         return true;
       } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + " is not stale because it's only " + deltaMs +
-              " ms old, and staleThresholdMs = " + staleThresholdMs);
-        }
+        LOG.trace("{} is not stale because it's only {} ms old "
+            + "and staleThresholdMs={}",  this, deltaMs, staleThresholdMs);
         return false;
       }
     }
@@ -194,13 +188,8 @@ public class ShortCircuitReplica {
       return false;
     }
     boolean result = slot.addAnchor();
-    if (LOG.isTraceEnabled()) {
-      if (result) {
-        LOG.trace(this + ": added no-checksum anchor to slot " + slot);
-      } else {
-        LOG.trace(this + ": could not add no-checksum anchor to slot " + slot);
-      }
-    }
+    LOG.trace("{}: {} no-checksum anchor to slot {}",
+        this, result ? "added" : "could not add", slot);
     return result;
   }
 
@@ -263,9 +252,7 @@ public class ShortCircuitReplica {
         suffix += "  scheduling " + slot + " for later release.";
       }
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("closed " + this + suffix);
-    }
+    LOG.trace("closed {}{}", this, suffix);
   }
 
   public FileInputStream getDataStream() {
@@ -293,9 +280,7 @@ public class ShortCircuitReplica {
       FileChannel channel = dataStream.getChannel();
       MappedByteBuffer mmap = channel.map(MapMode.READ_ONLY, 0, 
           Math.min(Integer.MAX_VALUE, channel.size()));
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": created mmap of size " + channel.size());
-      }
+      LOG.trace("{}: created mmap of size {}", this, channel.size());
       return mmap;
     } catch (IOException e) {
       LOG.warn(this + ": mmap error", e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
index 78325a3..fa40c15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
@@ -484,13 +484,9 @@ public class ShortCircuitShm {
         POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, mmappedLength);
     this.slots = new Slot[mmappedLength / BYTES_PER_SLOT];
     this.allocatedSlots = new BitSet(slots.length);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("creating " + this.getClass().getSimpleName() +
-          "(shmId=" + shmId +
-          ", mmappedLength=" + mmappedLength +
-          ", baseAddress=" + String.format("%x", baseAddress) +
-          ", slots.length=" + slots.length + ")");
-    }
+    LOG.trace("creating {}(shmId={}, mmappedLength={}, baseAddress={}, "
+        + "slots.length={})", this.getClass().getSimpleName(), shmId,
+        mmappedLength, String.format("%x", baseAddress), slots.length);
   }
 
   public final ShmId getShmId() {
@@ -615,9 +611,7 @@ public class ShortCircuitShm {
         "tried to unregister slot " + slotIdx + ", which was not registered.");
     allocatedSlots.set(slotIdx, false);
     slots[slotIdx] = null;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": unregisterSlot " + slotIdx);
-    }
+    LOG.trace("{}: unregisterSlot {}", this, slotIdx);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
index a9adb7e..e361252 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
@@ -36,18 +36,6 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public abstract class ByteArrayManager {
   static final Logger LOG = LoggerFactory.getLogger(ByteArrayManager.class);
-  private static final ThreadLocal<StringBuilder> DEBUG_MESSAGE =
-      new ThreadLocal<StringBuilder>() {
-    protected StringBuilder initialValue() {
-      return new StringBuilder();
-    }
-  };
-
-  private static void logDebugMessage() {
-    final StringBuilder b = DEBUG_MESSAGE.get();
-    LOG.debug(b.toString());
-    b.setLength(0);
-  }
 
   static final int MIN_ARRAY_LENGTH = 32;
   static final byte[] EMPTY_BYTE_ARRAY = {};
@@ -160,27 +148,18 @@ public abstract class ByteArrayManager {
      * via the {@link FixedLengthManager#recycle(byte[])} method.
      */
     synchronized byte[] allocate() throws InterruptedException {
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", ").append(this);
-      }
+      LOG.debug(", {}", this);
       for(; numAllocated >= maxAllocated;) {
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(": wait ...");
-          logDebugMessage();
-        }
+        LOG.debug(": wait ...");
 
         wait();
 
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append("wake up: ").append(this);
-        }
+        LOG.debug("wake up: {}", this);
       }
       numAllocated++;
 
       final byte[] array = freeQueue.poll();
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", recycled? ").append(array != null);
-      }
+      LOG.debug(", recycled? {}", array != null);
       return array != null? array : new byte[byteArrayLength];
     }
 
@@ -194,9 +173,7 @@ public abstract class ByteArrayManager {
     synchronized int recycle(byte[] array) {
       Preconditions.checkNotNull(array);
       Preconditions.checkArgument(array.length == byteArrayLength);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", ").append(this);
-      }
+      LOG.debug(", {}", this);
 
       notify();
       numAllocated--;
@@ -207,9 +184,7 @@ public abstract class ByteArrayManager {
       }
 
       if (freeQueue.size() < maxAllocated - numAllocated) {
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(", freeQueue.offer");
-        }
+        LOG.debug(", freeQueue.offer");
         freeQueue.offer(array);
       }
       return freeQueue.size();
@@ -349,9 +324,7 @@ public abstract class ByteArrayManager {
     public byte[] newByteArray(final int arrayLength)
         throws InterruptedException {
       Preconditions.checkArgument(arrayLength >= 0);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append("allocate(").append(arrayLength).append(")");
-      }
+      LOG.debug("allocate({})", arrayLength);
 
       final byte[] array;
       if (arrayLength == 0) {
@@ -365,18 +338,12 @@ public abstract class ByteArrayManager {
         final FixedLengthManager manager =
             managers.get(powerOfTwo, aboveThreshold);
 
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(": count=").append(count)
-              .append(aboveThreshold? ", aboveThreshold": ", belowThreshold");
-        }
+        LOG.debug(": count={}, {}Threshold", count,
+            aboveThreshold ? "above" : "below");
         array = manager != null? manager.allocate(): new byte[powerOfTwo];
       }
 
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", return byte[")
-            .append(array.length).append("]");
-        logDebugMessage();
-      }
+      LOG.debug(", return byte[{}]", array.length);
       return array;
     }
 
@@ -391,10 +358,7 @@ public abstract class ByteArrayManager {
     @Override
     public int release(final byte[] array) {
       Preconditions.checkNotNull(array);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get()
-            .append("recycle: array.length=").append(array.length);
-      }
+      LOG.debug("recycle: array.length={}", array.length);
 
       final int freeQueueSize;
       if (array.length == 0) {
@@ -404,10 +368,7 @@ public abstract class ByteArrayManager {
         freeQueueSize = manager == null? -1: manager.recycle(array);
       }
 
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", freeQueueSize=").append(freeQueueSize);
-        logDebugMessage();
-      }
+      LOG.debug(", freeQueueSize={}", freeQueueSize);
       return freeQueueSize;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
index a864d37..870103e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
@@ -134,9 +134,7 @@ final class TokenAspect<T extends FileSystem & Renewable> {
       if (token != null) {
         fs.setDelegationToken(token);
         addRenewAction(fs);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Created new DT for {}", token.getService());
-        }
+        LOG.debug("Created new DT for {}", token.getService());
       }
       hasInitedToken = true;
     }
@@ -149,9 +147,7 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   synchronized void initDelegationToken(UserGroupInformation ugi) {
     Token<?> token = selectDelegationToken(ugi);
     if (token != null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Found existing DT for {}", token.getService());
-      }
+      LOG.debug("Found existing DT for {}", token.getService());
       fs.setDelegationToken(token);
       hasInitedToken = true;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
index 4c23241..be5f17d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
@@ -182,9 +182,7 @@ public class URLConnectionFactory {
   public URLConnection openConnection(URL url, boolean isSpnego)
       throws IOException, AuthenticationException {
     if (isSpnego) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("open AuthenticatedURL connection {}", url);
-      }
+      LOG.debug("open AuthenticatedURL connection {}", url);
       UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
       final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
       return new AuthenticatedURL(new KerberosUgiAuthenticator(),


[51/58] [abbrv] hadoop git commit: Revert "HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai."

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index e122748..e245d2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -233,12 +233,16 @@ public class WebHdfsFileSystem extends FileSystem
       // refetch tokens.  even if ugi has credentials, don't attempt
       // to get another token to match hdfs/rpc behavior
       if (token != null) {
-        LOG.debug("Using UGI token: {}", token);
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Using UGI token: {}", token);
+        }
         canRefreshDelegationToken = false;
       } else {
         token = getDelegationToken(null);
         if (token != null) {
-          LOG.debug("Fetched new token: {}", token);
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Fetched new token: {}", token);
+          }
         } else { // security is disabled
           canRefreshDelegationToken = false;
         }
@@ -253,7 +257,9 @@ public class WebHdfsFileSystem extends FileSystem
     boolean replaced = false;
     if (canRefreshDelegationToken) {
       Token<?> token = getDelegationToken(null);
-      LOG.debug("Replaced expired token: {}", token);
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Replaced expired token: {}", token);
+      }
       setDelegationToken(token);
       replaced = (token != null);
     }
@@ -436,7 +442,9 @@ public class WebHdfsFileSystem extends FileSystem
     InetSocketAddress nnAddr = getCurrentNNAddr();
     final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
-    LOG.trace("url={}", url);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("url={}", url);
+    }
     return url;
   }
 
@@ -471,7 +479,9 @@ public class WebHdfsFileSystem extends FileSystem
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", parameters);
     final URL url = getNamenodeURL(path, query);
-    LOG.trace("url={}", url);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("url={}", url);
+    }
     return url;
   }
 
@@ -759,7 +769,9 @@ public class WebHdfsFileSystem extends FileSystem
       } catch (Exception e) { // catch json parser errors
         final IOException ioe =
             new IOException("Response decoding failure: "+e.toString(), e);
-        LOG.debug("Response decoding failure.", e);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Response decoding failure: {}", e.toString(), e);
+        }
         throw ioe;
       } finally {
         conn.disconnect();
@@ -1230,7 +1242,9 @@ public class WebHdfsFileSystem extends FileSystem
         cancelDelegationToken(delegationToken);
       }
     } catch (IOException ioe) {
-      LOG.debug("Token cancel failed: ", ioe);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Token cancel failed: ", ioe);
+      }
     } finally {
       super.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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 dfd0b57..7b62b97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1000,9 +1000,6 @@ Release 2.8.0 - UNRELEASED
     HDFS-8696. Make the lower and higher watermark in the DN Netty server
     configurable. (Xiaobing Zhou via wheat9)
 
-    HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client
-    package. (Mingliang Liu via wheat9)
-
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[53/58] [abbrv] hadoop git commit: HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client package. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index e245d2a..e122748 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -233,16 +233,12 @@ public class WebHdfsFileSystem extends FileSystem
       // refetch tokens.  even if ugi has credentials, don't attempt
       // to get another token to match hdfs/rpc behavior
       if (token != null) {
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("Using UGI token: {}", token);
-        }
+        LOG.debug("Using UGI token: {}", token);
         canRefreshDelegationToken = false;
       } else {
         token = getDelegationToken(null);
         if (token != null) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Fetched new token: {}", token);
-          }
+          LOG.debug("Fetched new token: {}", token);
         } else { // security is disabled
           canRefreshDelegationToken = false;
         }
@@ -257,9 +253,7 @@ public class WebHdfsFileSystem extends FileSystem
     boolean replaced = false;
     if (canRefreshDelegationToken) {
       Token<?> token = getDelegationToken(null);
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Replaced expired token: {}", token);
-      }
+      LOG.debug("Replaced expired token: {}", token);
       setDelegationToken(token);
       replaced = (token != null);
     }
@@ -442,9 +436,7 @@ public class WebHdfsFileSystem extends FileSystem
     InetSocketAddress nnAddr = getCurrentNNAddr();
     final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("url={}", url);
-    }
+    LOG.trace("url={}", url);
     return url;
   }
 
@@ -479,9 +471,7 @@ public class WebHdfsFileSystem extends FileSystem
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", parameters);
     final URL url = getNamenodeURL(path, query);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("url={}", url);
-    }
+    LOG.trace("url={}", url);
     return url;
   }
 
@@ -769,9 +759,7 @@ public class WebHdfsFileSystem extends FileSystem
       } catch (Exception e) { // catch json parser errors
         final IOException ioe =
             new IOException("Response decoding failure: "+e.toString(), e);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Response decoding failure: {}", e.toString(), e);
-        }
+        LOG.debug("Response decoding failure.", e);
         throw ioe;
       } finally {
         conn.disconnect();
@@ -1242,9 +1230,7 @@ public class WebHdfsFileSystem extends FileSystem
         cancelDelegationToken(delegationToken);
       }
     } catch (IOException ioe) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Token cancel failed: ", ioe);
-      }
+      LOG.debug("Token cancel failed: ", ioe);
     } finally {
       super.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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 7b62b97..dfd0b57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1000,6 +1000,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8696. Make the lower and higher watermark in the DN Netty server
     configurable. (Xiaobing Zhou via wheat9)
 
+    HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client
+    package. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[52/58] [abbrv] hadoop git commit: Revert "HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai."

Posted by zh...@apache.org.
Revert "HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai."

This reverts commit d5a9a3daa0224249221ffa7b8bd5751ab2feca56.


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

Branch: refs/heads/HDFS-7285
Commit: 6ee0539ede78b640f01c5eac18ded161182a7835
Parents: d5a9a3d
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 29 17:51:15 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 29 17:51:15 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 114 +++++++++++++------
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  77 +++++++++----
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  20 +++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  36 ++++--
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  85 +++++++++-----
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  36 +++---
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  47 +++++---
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   4 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  25 ++--
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  18 ++-
 .../protocol/datatransfer/PacketReceiver.java   |   8 +-
 .../hdfs/protocol/datatransfer/Sender.java      |   6 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java |   8 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  10 +-
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  45 +++++---
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 101 ++++++++++++----
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  33 ++++--
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  14 ++-
 .../hadoop/hdfs/util/ByteArrayManager.java      |  63 ++++++++--
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |   8 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |   4 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  28 +++--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 -
 24 files changed, 573 insertions(+), 230 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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
index f249692..4f37090 100644
--- 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
@@ -349,13 +349,17 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       if (clientContext.getUseLegacyBlockReaderLocal()) {
         reader = getLegacyBlockReaderLocal();
         if (reader != null) {
-          LOG.trace("{}: returning new legacy block reader local.", this);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": returning new legacy block reader local.");
+          }
           return reader;
         }
       } else {
         reader = getBlockReaderLocal();
         if (reader != null) {
-          LOG.trace("{}: returning new block reader local.", this);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": returning new block reader local.");
+          }
           return reader;
         }
       }
@@ -363,8 +367,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (scConf.isDomainSocketDataTraffic()) {
       reader = getRemoteBlockReaderFromDomain();
       if (reader != null) {
-        LOG.trace("{}: returning new remote block reader using UNIX domain "
-            + "socket on {}", this, pathInfo.getPath());
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + ": returning new remote block reader using " +
+              "UNIX domain socket on " + pathInfo.getPath());
+        }
         return reader;
       }
     }
@@ -399,8 +405,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             setVisibleLength(visibleLength).
             build();
         if (accessor == null) {
-          LOG.trace("{}: No ReplicaAccessor created by {}",
-              this, cls.getName());
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": No ReplicaAccessor created by " +
+                cls.getName());
+          }
         } else {
           return new ExternalBlockReader(accessor, visibleLength, startOffset);
         }
@@ -419,10 +427,14 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * first introduced in HDFS-2246.
    */
   private BlockReader getLegacyBlockReaderLocal() throws IOException {
-    LOG.trace("{}: trying to construct BlockReaderLocalLegacy", this);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
+    }
     if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
-      LOG.trace("{}: can't construct BlockReaderLocalLegacy because the address"
-          + "{} is not local", this, inetSocketAddress);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
+            "the address " + inetSocketAddress + " is not local");
+      }
       return null;
     }
     if (clientContext.getDisableLegacyBlockReaderLocal()) {
@@ -458,8 +470,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   }
 
   private BlockReader getBlockReaderLocal() throws InvalidToken {
-    LOG.trace("{}: trying to construct a BlockReaderLocal for short-circuit "
-        + " reads.", this);
+    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());
@@ -474,8 +488,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
     InvalidToken exc = info.getInvalidTokenException();
     if (exc != null) {
-      LOG.trace("{}: got InvalidToken exception while trying to construct "
-          + "BlockReaderLocal via {}", this, pathInfo.getPath());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": got InvalidToken exception while trying to " +
+            "construct BlockReaderLocal via " + pathInfo.getPath());
+      }
       throw exc;
     }
     if (info.getReplica() == null) {
@@ -511,7 +527,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
       if (info != null) return info;
     }
-    LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
+    }
     BlockReaderPeer curPeer;
     while (true) {
       curPeer = nextDomainPeer();
@@ -526,8 +544,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
             clientName);
         if (usedPeer.booleanValue()) {
-          LOG.trace("{}: allocShmSlot used up our previous socket {}.  "
-              + "Allocating a new one...", this, peer.getDomainSocket());
+          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;
@@ -542,7 +562,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         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.
-          LOG.debug("{}: closing stale domain peer {}", this, peer, e);
+          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.
@@ -595,7 +617,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
         if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
-          LOG.trace("Sending receipt verification byte for slot {}", slot);
+          LOG.trace("Sending receipt verification byte for slot " + slot);
           sock.getOutputStream().write(0);
         }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
@@ -628,7 +650,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       String msg = "access control error while " +
           "attempting to set up short-circuit access to " +
           fileName + resp.getMessage();
-      LOG.debug("{}:{}", this, msg);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(this + ":" + msg);
+      }
       return new ShortCircuitReplicaInfo(new InvalidToken(msg));
     default:
       LOG.warn(this + ": unknown response code " + resp.getStatus() +
@@ -660,8 +684,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
            " is not usable.", this, pathInfo);
       return null;
     }
-    LOG.trace("{}: trying to create a remote block reader from the UNIX domain "
-        + "socket at {}", this, pathInfo.getPath());
+    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();
@@ -675,15 +701,19 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       } catch (IOException ioe) {
         IOUtilsClient.cleanup(LOG, peer);
         if (isSecurityException(ioe)) {
-          LOG.trace("{}: got security exception while constructing a remote "
-                  + " block reader from the unix domain socket at {}",
-              this, pathInfo.getPath(), 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.
-          LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
+          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
@@ -717,8 +747,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    *             If there was another problem.
    */
   private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
-    LOG.trace("{}: trying to create a remote block reader from a TCP socket",
-        this);
+    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;
@@ -731,15 +763,19 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         return blockReader;
       } catch (IOException ioe) {
         if (isSecurityException(ioe)) {
-          LOG.trace("{}: got security exception while constructing a remote "
-              + "block reader from {}", this, peer, 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.
-          LOG.debug("Closed potentially stale remote peer {}", peer, ioe);
+          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);
@@ -772,7 +808,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, true);
       if (peer != null) {
-        LOG.trace("nextDomainPeer: reusing existing peer {}", peer);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("nextDomainPeer: reusing existing peer " + peer);
+        }
         return new BlockReaderPeer(peer, true);
       }
     }
@@ -794,18 +832,24 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, false);
       if (peer != null) {
-        LOG.trace("nextTcpPeer: reusing existing peer {}", peer);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("nextTcpPeer: reusing existing peer " + peer);
+        }
         return new BlockReaderPeer(peer, true);
       }
     }
     try {
       Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
         datanode);
-      LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("nextTcpPeer: created newConnectedPeer " + peer);
+      }
       return new BlockReaderPeer(peer, false);
     } catch (IOException e) {
-      LOG.trace("nextTcpPeer: failed to create newConnectedPeer connected to"
-          + "{}", datanode);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("nextTcpPeer: failed to create newConnectedPeer " +
+                  "connected to " + datanode);
+      }
       throw e;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index 10dc35c..62e7af6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@ -412,10 +412,17 @@ class BlockReaderLocal implements BlockReader {
   public synchronized int read(ByteBuffer buf) throws IOException {
     boolean canSkipChecksum = createNoChecksumContext();
     try {
-      String traceFormatStr = "read(buf.remaining={}, block={}, filename={}, "
-          + "canSkipChecksum={})";
-      LOG.trace(traceFormatStr + ": starting",
-          buf.remaining(), block, filename, canSkipChecksum);
+      String traceString = null;
+      if (LOG.isTraceEnabled()) {
+        traceString = new StringBuilder().
+            append("read(").
+            append("buf.remaining=").append(buf.remaining()).
+            append(", block=").append(block).
+            append(", filename=").append(filename).
+            append(", canSkipChecksum=").append(canSkipChecksum).
+            append(")").toString();
+        LOG.info(traceString + ": starting");
+      }
       int nRead;
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
@@ -424,12 +431,14 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(buf, canSkipChecksum);
         }
       } catch (IOException e) {
-        LOG.trace(traceFormatStr + ": I/O error",
-            buf.remaining(), block, filename, canSkipChecksum, e);
+        if (LOG.isTraceEnabled()) {
+          LOG.info(traceString + ": I/O error", e);
+        }
         throw e;
       }
-      LOG.trace(traceFormatStr + ": returning {}",
-          buf.remaining(), block, filename, canSkipChecksum, nRead);
+      if (LOG.isTraceEnabled()) {
+        LOG.info(traceString + ": returning " + nRead);
+      }
       return nRead;
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
@@ -481,8 +490,10 @@ class BlockReaderLocal implements BlockReader {
     }
     dataBuf.limit(dataBuf.position());
     dataBuf.position(Math.min(dataBuf.position(), slop));
-    LOG.trace("loaded {} bytes into bounce buffer from offset {} of {}",
-        dataBuf.remaining(), oldDataPos, block);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("loaded " + dataBuf.remaining() + " bytes into bounce " +
+          "buffer from offset " + oldDataPos + " of " + block);
+    }
     return dataBuf.limit() != maxReadaheadLength;
   }
 
@@ -554,10 +565,18 @@ class BlockReaderLocal implements BlockReader {
     boolean canSkipChecksum = createNoChecksumContext();
     int nRead;
     try {
-      final String traceFormatStr = "read(arr.length={}, off={}, len={}, "
-          + "filename={}, block={}, canSkipChecksum={})";
-      LOG.trace(traceFormatStr + ": starting",
-          arr.length, off, len, filename, block, canSkipChecksum);
+      String traceString = null;
+      if (LOG.isTraceEnabled()) {
+        traceString = new StringBuilder().
+            append("read(arr.length=").append(arr.length).
+            append(", off=").append(off).
+            append(", len=").append(len).
+            append(", filename=").append(filename).
+            append(", block=").append(block).
+            append(", canSkipChecksum=").append(canSkipChecksum).
+            append(")").toString();
+        LOG.trace(traceString + ": starting");
+      }
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
           nRead = readWithoutBounceBuffer(arr, off, len);
@@ -565,12 +584,14 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
         }
       } catch (IOException e) {
-        LOG.trace(traceFormatStr + ": I/O error",
-            arr.length, off, len, filename, block, canSkipChecksum, e);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(traceString + ": I/O error", e);
+        }
         throw e;
       }
-      LOG.trace(traceFormatStr + ": returning {}",
-          arr.length, off, len, filename, block, canSkipChecksum, nRead);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(traceString + ": returning " + nRead);
+      }
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
     }
@@ -613,9 +634,11 @@ class BlockReaderLocal implements BlockReader {
       dataBuf.position(dataBuf.position() + discardedFromBuf);
       remaining -= discardedFromBuf;
     }
-    LOG.trace("skip(n={}, block={}, filename={}): discarded {} bytes from "
-            + "dataBuf and advanced dataPos by {}",
-        n, block, filename, discardedFromBuf, remaining);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("skip(n=" + n + ", block=" + block + ", filename=" + 
+        filename + "): discarded " + discardedFromBuf + " bytes from " +
+        "dataBuf and advanced dataPos by " + remaining);
+    }
     dataPos += remaining;
     return n;
   }
@@ -630,7 +653,9 @@ class BlockReaderLocal implements BlockReader {
   public synchronized void close() throws IOException {
     if (closed) return;
     closed = true;
-    LOG.trace("close(filename={}, block={})", filename, block);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("close(filename=" + filename + ", block=" + block + ")");
+    }
     replica.unref();
     freeDataBufIfExists();
     freeChecksumBufIfExists();
@@ -680,9 +705,11 @@ class BlockReaderLocal implements BlockReader {
         (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
     if (anchor) {
       if (!createNoChecksumContext()) {
-        LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
-            + "given, we aren't skipping checksums, and the block is not "
-            + "mlocked.", block, filename);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("can't get an mmap for " + block + " of " + filename + 
+              " since SKIP_CHECKSUMS was not given, " +
+              "we aren't skipping checksums, and the block is not mlocked.");
+        }
         return null;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 4a1828e..5235287 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -221,9 +221,11 @@ class BlockReaderLocalLegacy implements BlockReader {
       File blkfile = new File(pathinfo.getBlockPath());
       dataIn = new FileInputStream(blkfile);
 
-      LOG.debug("New BlockReaderLocalLegacy for file {} of size {} startOffset "
-              + "{} length {} short circuit checksum {}",
-          blkfile, blkfile.length(), startOffset, length, !skipChecksumCheck);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("New BlockReaderLocalLegacy for file " + blkfile + " of size "
+            + blkfile.length() + " startOffset " + startOffset + " length "
+            + length + " short circuit checksum " + !skipChecksumCheck);
+      }
 
       if (!skipChecksumCheck) {
         // get the metadata file
@@ -290,7 +292,9 @@ class BlockReaderLocalLegacy implements BlockReader {
       // channel for the DataNode to notify the client that the path has been
       // invalidated.  Therefore, our only option is to skip caching.
       if (pathinfo != null && !storageType.isTransient()) {
-        LOG.debug("Cached location of block {} as {}", blk, pathinfo);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Cached location of block " + blk + " as " + pathinfo);
+        }
         localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
       }
     } catch (IOException e) {
@@ -599,7 +603,9 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized int read(byte[] buf, int off, int len) throws IOException {
-    LOG.trace("read off {} len {}", off, len);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read off " + off + " len " + len);
+    }
     if (!verifyChecksum) {
       return dataIn.read(buf, off, len);
     }
@@ -618,7 +624,9 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized long skip(long n) throws IOException {
-    LOG.debug("skip {}", n);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("skip " + n);
+    }
     if (n <= 0) {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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
index 8f3df81..92d117c 100644
--- 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
@@ -419,7 +419,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     final int idx = r.nextInt(localInterfaceAddrs.length);
     final SocketAddress addr = localInterfaceAddrs[idx];
-    LOG.debug("Using local interface {}", addr);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using local interface " + addr);
+    }
     return addr;
   }
 
@@ -1214,7 +1216,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                              InetSocketAddress[] favoredNodes) throws IOException {
     checkOpen();
     final FsPermission masked = applyUMask(permission);
-    LOG.debug("{}: masked={}", src, masked);
+    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),
@@ -1811,8 +1815,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
               smallBufferSize));
           in = new DataInputStream(pair.in);
 
-          LOG.debug("write to {}: {}, block={}",
-              datanodes[j], Op.BLOCK_CHECKSUM, block);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("write to " + datanodes[j] + ": "
+                + Op.BLOCK_CHECKSUM + ", block=" + block);
+          }
           // get block MD5
           new Sender(out).blockChecksum(block, lb.getBlockToken());
 
@@ -1876,10 +1882,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           }
         } catch (InvalidBlockTokenException ibte) {
           if (i > lastRetriedIndex) {
-            LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
-                  + "for file {} for block {} from datanode {}. Will retry the "
-                  + "block once.",
-                src, block, datanodes[j]);
+            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
@@ -1933,7 +1941,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try {
       sock = socketFactory.createSocket();
       String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
-      LOG.debug("Connecting to datanode {}", dnAddr);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Connecting to datanode " + dnAddr);
+      }
       NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
       sock.setSoTimeout(timeout);
   
@@ -2553,7 +2563,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       absPermission = applyUMask(null);
     } 
 
-    LOG.debug("{}: masked={}", src, absPermission);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug(src + ": masked=" + absPermission);
+    }
     TraceScope scope = tracer.newScope("mkdir");
     try {
       return namenode.mkdirs(src, absPermission, createParent);
@@ -3049,7 +3061,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       }
     });
     HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
-    LOG.debug("Using hedged reads; pool threads={}", num);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using hedged reads; pool threads=" + num);
+    }
   }
 
   ThreadPoolExecutor getHedgedReadsThreadPool() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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
index 81e8c27..7101753 100644
--- 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
@@ -315,7 +315,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if (locatedBlocks == null || refresh) {
       newInfo = dfsClient.getLocatedBlocks(src, 0);
     }
-    DFSClient.LOG.debug("newInfo = {}", newInfo);
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("newInfo = " + newInfo);
+    }
     if (newInfo == null) {
       throw new IOException("Cannot open filename " + src);
     }
@@ -381,8 +383,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           replicaNotFoundCount--;
         }
         
-        DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
-              + " for block {}", datanode, locatedblock.getBlock(), ioe);
+        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);
@@ -1063,7 +1067,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     final String dnAddr =
         chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
-    DFSClient.LOG.debug("Connecting to datanode {}", dnAddr);
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+    }
     InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
     return new DNAddrPair(chosenNode, targetAddr, storageType);
   }
@@ -1303,8 +1309,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             future.get();
             return;
           }
-          DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
-              + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
+          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();
@@ -1331,8 +1340,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
               .submit(getFromDataNodeCallable);
           futures.add(oneMoreRequest);
         } catch (IOException ioe) {
-          DFSClient.LOG.debug("Failed getting node for hedged read: {}",
-              ioe.getMessage());
+          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.
@@ -1588,8 +1599,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             throw new IOException(errMsg);
           }
         } catch (IOException e) {//make following read to retry
-          DFSClient.LOG.debug("Exception while seek to {} from {} of {} from "
-              + "{}", targetPos, getCurrentBlock(), src, currentNode, e);
+          if(DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Exception while seek to " + targetPos
+                + " from " + getCurrentBlock() + " of " + src + " from "
+                + currentNode, e);
+          }
         }
       }
     }
@@ -1805,16 +1819,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     } else {
       length63 = 1 + curEnd - curPos;
       if (length63 <= 0) {
-        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {}"
-                + " of {}; {} bytes left in block. blockPos={}; curPos={};"
-                + "curEnd={}",
-            curPos, src, length63, blockPos, curPos, curEnd);
+        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;
       }
-      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid going "
-              + "more than one byte past the end of the block.  blockPos={}; "
-              +" curPos={}; curEnd={}",
-          maxLength, length63, blockPos, curPos, curEnd);
+      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;
@@ -1828,20 +1846,28 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         // 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.
-        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {} "
-            + " of {}; 31-bit MappedByteBuffer limit exceeded.  blockPos={}, "
-            + "curEnd={}", curPos, src, blockPos, curEnd);
+        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;
-      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid 31-bit "
-          + "limit.  blockPos={}; curPos={}; curEnd={}",
-          maxLength, length, blockPos, curPos, curEnd);
+      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) {
-      DFSClient.LOG.debug("unable to perform a zero-copy read from offset {} of"
-          + " {}; BlockReader#getClientMmap returned null.", curPos, src);
+      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;
@@ -1855,8 +1881,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       synchronized (infoLock) {
         readStatistics.addZeroCopyBytes(length);
       }
-      DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
-          + "zero-copy read path.  blockEnd = {}", length, curPos, blockEnd);
+      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) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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
index 47c4b7e..7a40d73 100755
--- 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
@@ -190,9 +190,9 @@ public class DFSOutputStream extends FSOutputSummer
     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
     this.cachingStrategy = new AtomicReference<CachingStrategy>(
         dfsClient.getDefaultWriteCachingStrategy());
-    if (progress != null) {
-      DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
-          +"{}", src);
+    if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug(
+          "Set non-null progress callback on DFSOutputStream " + src);
     }
     
     this.bytesPerChecksum = checksum.getBytesPerChecksum();
@@ -365,9 +365,12 @@ public class DFSOutputStream extends FSOutputSummer
     final int chunkSize = csize + getChecksumSize();
     chunksPerPacket = Math.max(bodySize/chunkSize, 1);
     packetSize = chunkSize*chunksPerPacket;
-    DFSClient.LOG.debug("computePacketChunkSize: src={}, chunkSize={}, "
-            + "chunksPerPacket={}, packetSize={}",
-        src, chunkSize, chunksPerPacket, packetSize);
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("computePacketChunkSize: src=" + src +
+                ", chunkSize=" + chunkSize +
+                ", chunksPerPacket=" + chunksPerPacket +
+                ", packetSize=" + packetSize);
+    }
   }
 
   protected TraceScope createWriteTraceScope() {
@@ -394,10 +397,14 @@ public class DFSOutputStream extends FSOutputSummer
     if (currentPacket == null) {
       currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
           .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
-      DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno={},"
-              + " src={}, packetSize={}, chunksPerPacket={}, bytesCurBlock={}",
-          currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
-          getStreamer().getBytesCurBlock());
+      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);
@@ -551,9 +558,12 @@ public class DFSOutputStream extends FSOutputSummer
         int numKept = flushBuffer(!endBlock, true);
         // bytesCurBlock potentially incremented if there was buffered data
 
-        DFSClient.LOG.debug("DFSClient flush():  bytesCurBlock={}, "
-                + "lastFlushOffset={}, createNewBlock={}",
-            getStreamer().getBytesCurBlock(), lastFlushOffset, endBlock);
+        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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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 71ce7cc..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
@@ -462,13 +462,19 @@ public class DFSUtilClient {
     InetAddress addr = targetAddr.getAddress();
     Boolean cached = localAddrMap.get(addr.getHostAddress());
     if (cached != null) {
-      LOG.trace("Address {} is {} local", targetAddr, (cached ? "" : "not"));
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Address " + targetAddr +
+            (cached ? " is local" : " is not local"));
+      }
       return cached;
     }
 
     boolean local = NetUtils.isLocalAddress(addr);
 
-    LOG.trace("Address {} is {} local", targetAddr, (local ? "" : "not"));
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Address " + targetAddr +
+          (local ? " is local" : " is not local"));
+    }
     localAddrMap.put(addr.getHostAddress(), local);
     return local;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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
index 4ea1f41..d1d8d37 100644
--- 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
@@ -132,14 +132,18 @@ class DataStreamer extends Daemon {
       final int length, final DFSClient client) throws IOException {
     final DfsClientConf conf = client.getConf();
     final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
-    LOG.debug("Connecting to datanode {}", dnAddr);
+    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);
-    LOG.debug("Send buf size {}", sock.getSendBufferSize());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Send buf size " + sock.getSendBufferSize());
+    }
     return sock;
   }
 
@@ -480,7 +484,9 @@ class DataStreamer extends Daemon {
   }
 
   private void endBlock() {
-    LOG.debug("Closing old block {}", block);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Closing old block " + block);
+    }
     this.setName("DataStreamer for file " + src);
     closeResponder();
     closeStream();
@@ -561,11 +567,15 @@ class DataStreamer extends Daemon {
 
         // get new block from namenode.
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          LOG.debug("Allocating new block");
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Allocating new block");
+          }
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          LOG.debug("Append to block {}", block);
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Append to block " + block);
+          }
           setupPipelineForAppendOrRecovery();
           if (streamerClosed) {
             continue;
@@ -617,7 +627,10 @@ class DataStreamer extends Daemon {
           }
         }
 
-        LOG.debug("DataStreamer block {} sending packet {}", block, one);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("DataStreamer block " + block +
+              " sending packet " + one);
+        }
 
         // write out data to remote datanode
         TraceScope writeScope = dfsClient.getTracer().
@@ -728,7 +741,9 @@ class DataStreamer extends Daemon {
     TraceScope scope = dfsClient.getTracer().
         newScope("waitForAckedSeqno");
     try {
-      LOG.debug("Waiting for ack for: {}", seqno);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Waiting for ack for: " + seqno);
+      }
       long begin = Time.monotonicNow();
       try {
         synchronized (dataQueue) {
@@ -940,8 +955,8 @@ class DataStreamer extends Daemon {
             LOG.warn("Slow ReadProcessor read fields took " + duration
                 + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
                 + ack + ", targets: " + Arrays.asList(targets));
-          } else {
-            LOG.debug("DFSClient {}", ack);
+          } else if (LOG.isDebugEnabled()) {
+            LOG.debug("DFSClient " + ack);
           }
 
           long seqno = ack.getSeqno();
@@ -1161,7 +1176,9 @@ class DataStreamer extends Daemon {
   }
 
   private void addDatanode2ExistingPipeline() throws IOException {
-    DataTransferProtocol.LOG.debug("lastAckedSeqno = {}", lastAckedSeqno);
+    if (DataTransferProtocol.LOG.isDebugEnabled()) {
+      DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
+    }
       /*
        * Is data transfer necessary?  We have the following cases.
        *
@@ -1628,8 +1645,10 @@ class DataStreamer extends Daemon {
           new HashSet<String>(Arrays.asList(favoredNodes));
       for (int i = 0; i < nodes.length; i++) {
         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
-        LOG.debug("{} was chosen by name node (favored={}).",
-            nodes[i].getXferAddrWithHostname(), pinnings[i]);
+        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.
@@ -1768,7 +1787,9 @@ class DataStreamer extends Daemon {
       packet.addTraceParent(Tracer.getCurrentSpanId());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
-      LOG.debug("Queued packet {}", packet.getSeqno());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Queued packet " + packet.getSeqno());
+      }
       dataQueue.notifyAll();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index 017be9f..15a5bee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -250,7 +250,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       PacketHeader header = new PacketHeader();
       header.readFields(in);
 
-      LOG.debug("DFSClient readChunk got header {}", header);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("DFSClient readChunk got header " + header);
+      }
 
       // Sanity check the lengths
       if (!header.sanityCheck(lastSeqNo)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index ca31e67..7a7932d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -135,9 +135,14 @@ public class RemoteBlockReader2  implements BlockReader {
   @Override
   public synchronized int read(byte[] buf, int off, int len) 
                                throws IOException {
-    UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
-    LOG.trace("Starting read #{} file {} from datanode {}",
-        randomId, filename, datanodeID.getHostName());
+
+    UUID randomId = null;
+    if (LOG.isTraceEnabled()) {
+      randomId = UUID.randomUUID();
+      LOG.trace(String.format("Starting read #%s file %s from datanode %s",
+        randomId.toString(), this.filename,
+        this.datanodeID.getHostName()));
+    }
 
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
       TraceScope scope = tracer.newScope(
@@ -149,7 +154,9 @@ public class RemoteBlockReader2  implements BlockReader {
       }
     }
 
-    LOG.trace("Finishing read #{}", randomId);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(String.format("Finishing read #" + randomId));
+    }
 
     if (curDataSlice.remaining() == 0) {
       // we're at EOF now
@@ -196,7 +203,9 @@ public class RemoteBlockReader2  implements BlockReader {
     curDataSlice = packetReceiver.getDataSlice();
     assert curDataSlice.capacity() == curHeader.getDataLen();
     
-    LOG.trace("DFSClient readNextPacket got header {}", curHeader);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("DFSClient readNextPacket got header " + curHeader);
+    }
 
     // Sanity check the lengths
     if (!curHeader.sanityCheck(lastSeqNo)) {
@@ -267,8 +276,10 @@ public class RemoteBlockReader2  implements BlockReader {
   }
 
   private void readTrailingEmptyPacket() throws IOException {
-    LOG.trace("Reading empty packet at end of read");
-
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Reading empty packet at end of read");
+    }
+    
     packetReceiver.receiveNextPacket(in);
 
     PacketHeader trailer = packetReceiver.getHeader();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/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
index 8457d65..c3d2cfc 100644
--- 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
@@ -308,7 +308,10 @@ public class LeaseRenewer {
               }
               LeaseRenewer.this.run(id);
             } catch(InterruptedException e) {
-              LOG.debug("LeaseRenewer is interrupted.", e);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
+                    + " is interrupted.", e);
+              }
             } finally {
               synchronized(LeaseRenewer.this) {
                 Factory.INSTANCE.remove(LeaseRenewer.this);
@@ -396,7 +399,9 @@ public class LeaseRenewer {
     }
 
     if (daemonCopy != null) {
-      LOG.debug("Wait for lease checker to terminate");
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Wait for lease checker to terminate");
+      }
       daemonCopy.join();
     }
   }
@@ -419,11 +424,16 @@ public class LeaseRenewer {
       //skip if current client name is the same as the previous name.
       if (!c.getClientName().equals(previousName)) {
         if (!c.renewLease()) {
-          LOG.debug("Did not renew lease for client {}", c);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Did not renew lease for client " +
+                c);
+          }
           continue;
         }
         previousName = c.getClientName();
-        LOG.debug("Lease renewed for client {}", previousName);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Lease renewed for client " + previousName);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
index e6709d9..c4093b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
@@ -147,9 +147,11 @@ public class PacketReceiver implements Closeable {
       throw new IOException("Invalid header length " + headerLen);
     }
     
-    LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
-        dataPlusChecksumLen, headerLen);
-
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("readNextPacket: dataPlusChecksumLen = " + dataPlusChecksumLen +
+          " headerLen = " + headerLen);
+    }
+    
     // Sanity check the buffer size so we don't allocate too much memory
     // and OOME.
     int totalLen = payloadLen + headerLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index d2bc348..e856211 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -73,8 +73,10 @@ public class Sender implements DataTransferProtocol {
 
   private static void send(final DataOutputStream out, final Op opcode,
       final Message proto) throws IOException {
-    LOG.trace("Sending DataTransferOp {}: {}",
-        proto.getClass().getSimpleName(), proto);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Sending DataTransferOp " + proto.getClass().getSimpleName()
+          + ": " + proto);
+    }
     op(out, opcode);
     proto.writeDelimitedTo(out);
     out.flush();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
index 006d304..256caff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
@@ -332,9 +332,11 @@ public final class DataTransferSaslUtil {
   public static IOStreamPair createStreamPair(Configuration conf,
       CipherOption cipherOption, OutputStream out, InputStream in, 
       boolean isServer) throws IOException {
-    LOG.debug("Creating IOStreamPair of CryptoInputStream and "
-        + "CryptoOutputStream.");
-    CryptoCodec codec = CryptoCodec.getInstance(conf,
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating IOStreamPair of CryptoInputStream and " +
+          "CryptoOutputStream.");
+    }
+    CryptoCodec codec = CryptoCodec.getInstance(conf, 
         cipherOption.getCipherSuite());
     byte[] inKey = cipherOption.getInKey();
     byte[] inIv = cipherOption.getInIv();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index 24e1dd2..f764275 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -130,7 +130,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
+    }
     rpcProxy = createClientDatanodeProtocolProxy(addr,
         UserGroupInformation.getCurrentUser(), conf,
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
@@ -141,8 +143,10 @@ public class ClientDatanodeProtocolTranslatorPB implements
       boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
-
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
+    }
+    
     // Since we're creating a new UserGroupInformation here, we know that no
     // future RPC proxies will be able to re-use the same connection. And
     // usages of this proxy tend to be one-off calls.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
index 4ffc108..f70398a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
@@ -129,13 +129,18 @@ public class DfsClientShmManager implements Closeable {
       ShmId shmId = shm.getShmId();
       Slot slot = shm.allocAndRegisterSlot(blockId);
       if (shm.isFull()) {
-        LOG.trace("{}: pulled the last slot {} out of {}",
-            this, slot.getSlotIdx(), shm);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + ": pulled the last slot " + slot.getSlotIdx() +
+              " out of " + shm);
+        }
         DfsClientShm removedShm = notFull.remove(shmId);
         Preconditions.checkState(removedShm == shm);
         full.put(shmId, shm);
       } else {
-        LOG.trace("{}: pulled slot {} out of {}", this, slot.getSlotIdx(), shm);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + ": pulled slot " + slot.getSlotIdx() +
+              " out of " + shm);
+        }
       }
       return slot;
     }
@@ -182,7 +187,9 @@ public class DfsClientShmManager implements Closeable {
           DfsClientShm shm = 
               new DfsClientShm(PBHelperClient.convert(resp.getId()),
                   fis[0], this, peer);
-          LOG.trace("{}: createNewShm: created {}", this, shm);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": createNewShm: created " + shm);
+          }
           return shm;
         } finally {
           try {
@@ -227,11 +234,15 @@ public class DfsClientShmManager implements Closeable {
         String clientName, ExtendedBlockId blockId) throws IOException {
       while (true) {
         if (closed) {
-          LOG.trace("{}: the DfsClientShmManager has been closed.", this);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": the DfsClientShmManager has been closed.");
+          }
           return null;
         }
         if (disabled) {
-          LOG.trace("{}: shared memory segment access is disabled.", this);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": shared memory segment access is disabled.");
+          }
           return null;
         }
         // Try to use an existing slot.
@@ -242,7 +253,9 @@ public class DfsClientShmManager implements Closeable {
         // There are no free slots.  If someone is loading more slots, wait
         // for that to finish.
         if (loading) {
-          LOG.trace("{}: waiting for loading to finish...", this);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": waiting for loading to finish...");
+          }
           finishedLoading.awaitUninterruptibly();
         } else {
           // Otherwise, load the slot ourselves.
@@ -269,9 +282,11 @@ public class DfsClientShmManager implements Closeable {
             // fired and marked the shm as disconnected.  In this case, we
             // obviously don't want to add the SharedMemorySegment to our list
             // of valid not-full segments.
-            LOG.debug("{}: the UNIX domain socket associated with this "
-                + "short-circuit memory closed before we could make use of "
-                + "the shm.", this);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(this + ": the UNIX domain socket associated with " +
+                  "this short-circuit memory closed before we could make " +
+                  "use of the shm.");
+            }
           } else {
             notFull.put(shm.getShmId(), shm);
           }
@@ -294,7 +309,9 @@ public class DfsClientShmManager implements Closeable {
         Preconditions.checkState(!full.containsKey(shm.getShmId()));
         Preconditions.checkState(!notFull.containsKey(shm.getShmId()));
         if (shm.isEmpty()) {
-          LOG.trace("{}: freeing empty stale {}", this, shm);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": freeing empty stale " + shm);
+          }
           shm.free();
         }
       } else {
@@ -319,8 +336,10 @@ public class DfsClientShmManager implements Closeable {
           // lowest ID, but it could still occur.  In most workloads,
           // fragmentation should not be a major concern, since it doesn't impact
           // peak file descriptor usage or the speed of allocation.
-          LOG.trace("{}: shutting down UNIX domain socket for empty {}",
-              this, shm);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": shutting down UNIX domain socket for " +
+                "empty " + shm);
+          }
           shutdown(shm);
         } else {
           notFull.put(shmId, shm);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
index 07f5064..52c1a6e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
@@ -103,7 +103,9 @@ public class ShortCircuitCache implements Closeable {
         if (ShortCircuitCache.this.closed) return;
         long curMs = Time.monotonicNow();
 
-        LOG.debug("{}: cache cleaner running at {}", this, curMs);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(this + ": cache cleaner running at " + curMs);
+        }
 
         int numDemoted = demoteOldEvictableMmaped(curMs);
         int numPurged = 0;
@@ -125,9 +127,11 @@ public class ShortCircuitCache implements Closeable {
           numPurged++;
         }
 
-        LOG.debug("{}: finishing cache cleaner run started at {}. Demoted {} "
-            + "mmapped replicas; purged {} replicas.",
-            this, curMs, numDemoted, numPurged);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(this + ": finishing cache cleaner run started at " +
+            curMs + ".  Demoted " + numDemoted + " mmapped replicas; " +
+            "purged " + numPurged + " replicas.");
+        }
       } finally {
         ShortCircuitCache.this.lock.unlock();
       }
@@ -182,7 +186,9 @@ public class ShortCircuitCache implements Closeable {
 
     @Override
     public void run() {
-      LOG.trace("{}: about to release {}", ShortCircuitCache.this, slot);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(ShortCircuitCache.this + ": about to release " + slot);
+      }
       final DfsClientShm shm = (DfsClientShm)slot.getShm();
       final DomainSocket shmSock = shm.getPeer().getDomainSocket();
       final String path = shmSock.getPath();
@@ -199,7 +205,9 @@ public class ShortCircuitCache implements Closeable {
           String error = resp.hasError() ? resp.getError() : "(unknown)";
           throw new IOException(resp.getStatus().toString() + ": " + error);
         }
-        LOG.trace("{}: released {}", this, slot);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(ShortCircuitCache.this + ": released " + slot);
+        }
         success = true;
       } catch (IOException e) {
         LOG.error(ShortCircuitCache.this + ": failed to release " +
@@ -425,7 +433,9 @@ public class ShortCircuitCache implements Closeable {
           purgeReason = "purging replica because it is stale.";
         }
         if (purgeReason != null) {
-          LOG.debug("{}: {}", this, purgeReason);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(this + ": " + purgeReason);
+          }
           purge(replica);
         }
       }
@@ -667,8 +677,10 @@ public class ShortCircuitCache implements Closeable {
       ShortCircuitReplicaInfo info = null;
       do {
         if (closed) {
-          LOG.trace("{}: can't fethchOrCreate {} because the cache is closed.",
-              this, key);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": can't fetchOrCreate " + key +
+                " because the cache is closed.");
+          }
           return null;
         }
         Waitable<ShortCircuitReplicaInfo> waitable = replicaInfoMap.get(key);
@@ -676,7 +688,9 @@ public class ShortCircuitCache implements Closeable {
           try {
             info = fetch(key, waitable);
           } catch (RetriableException e) {
-            LOG.debug("{}: retrying {}", this, e.getMessage());
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(this + ": retrying " + e.getMessage());
+            }
             continue;
           }
         }
@@ -707,7 +721,9 @@ public class ShortCircuitCache implements Closeable {
     // ShortCircuitReplica.  So we simply wait for it to complete.
     ShortCircuitReplicaInfo info;
     try {
-      LOG.trace("{}: found waitable for {}", this, key);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": found waitable for " + key);
+      }
       info = waitable.await();
     } catch (InterruptedException e) {
       LOG.info(this + ": interrupted while waiting for " + key);
@@ -749,7 +765,9 @@ public class ShortCircuitCache implements Closeable {
     // Handle loading a new replica.
     ShortCircuitReplicaInfo info = null;
     try {
-      LOG.trace("{}: loading {}", this, key);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": loading " + key);
+      }
       info = creator.createShortCircuitReplicaInfo();
     } catch (RuntimeException e) {
       LOG.warn(this + ": failed to load " + key, e);
@@ -759,7 +777,9 @@ public class ShortCircuitCache implements Closeable {
     try {
       if (info.getReplica() != null) {
         // On success, make sure the cache cleaner thread is running.
-        LOG.trace("{}: successfully loaded {}", this, info.getReplica());
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + ": successfully loaded " + info.getReplica());
+        }
         startCacheCleanerThreadIfNeeded();
         // Note: new ShortCircuitReplicas start with a refCount of 2,
         // indicating that both this cache and whoever requested the 
@@ -791,8 +811,10 @@ public class ShortCircuitCache implements Closeable {
           cleanerExecutor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
               TimeUnit.MILLISECONDS);
       cacheCleaner.setFuture(future);
-      LOG.debug("{}: starting cache cleaner thread which will run every {} ms",
-          this, rateMs);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(this + ": starting cache cleaner thread which will run " +
+          "every " + rateMs + " ms");
+      }
     }
   }
 
@@ -810,12 +832,17 @@ public class ShortCircuitCache implements Closeable {
           long lastAttemptTimeMs = (Long)replica.mmapData;
           long delta = Time.monotonicNow() - lastAttemptTimeMs;
           if (delta < mmapRetryTimeoutMs) {
-            LOG.trace("{}: can't create client mmap for {} because we failed to"
-                + " create one just {}ms ago.", this, replica, delta);
+            if (LOG.isTraceEnabled()) {
+              LOG.trace(this + ": can't create client mmap for " +
+                  replica + " because we failed to " +
+                  "create one just " + delta + "ms ago.");
+            }
             return null;
           }
-          LOG.trace("{}: retrying client mmap for {}, {} ms after the previous "
-              + "failure.", this, replica, delta);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this + ": retrying client mmap for " + replica +
+                ", " + delta + " ms after the previous failure.");
+          }
         } else if (replica.mmapData instanceof Condition) {
           Condition cond = (Condition)replica.mmapData;
           cond.awaitUninterruptibly();
@@ -938,10 +965,38 @@ public class ShortCircuitCache implements Closeable {
           }
         }
       }
-      LOG.debug("visiting {} with outstandingMmapCount={}, replicas={}, "
-          + "failedLoads={}, evictable={}, evictableMmapped={}",
-          visitor.getClass().getName(), outstandingMmapCount, replicas,
-          failedLoads, evictable, evictableMmapped);
+      if (LOG.isDebugEnabled()) {
+        StringBuilder builder = new StringBuilder();
+        builder.append("visiting ").append(visitor.getClass().getName()).
+            append("with outstandingMmapCount=").append(outstandingMmapCount).
+            append(", replicas=");
+        String prefix = "";
+        for (Entry<ExtendedBlockId, ShortCircuitReplica> entry : replicas.entrySet()) {
+          builder.append(prefix).append(entry.getValue());
+          prefix = ",";
+        }
+        prefix = "";
+        builder.append(", failedLoads=");
+        for (Entry<ExtendedBlockId, InvalidToken> entry : failedLoads.entrySet()) {
+          builder.append(prefix).append(entry.getValue());
+          prefix = ",";
+        }
+        prefix = "";
+        builder.append(", evictable=");
+        for (Entry<Long, ShortCircuitReplica> entry : evictable.entrySet()) {
+          builder.append(prefix).append(entry.getKey()).
+              append(":").append(entry.getValue());
+          prefix = ",";
+        }
+        prefix = "";
+        builder.append(", evictableMmapped=");
+        for (Entry<Long, ShortCircuitReplica> entry : evictableMmapped.entrySet()) {
+          builder.append(prefix).append(entry.getKey()).
+              append(":").append(entry.getValue());
+          prefix = ",";
+        }
+        LOG.debug(builder.toString());
+      }
       visitor.visit(outstandingMmapCount, replicas, failedLoads,
             evictable, evictableMmapped);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
index 38cf22b..37566e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
@@ -154,19 +154,25 @@ public class ShortCircuitReplica {
       // Check staleness by looking at the shared memory area we use to
       // communicate with the DataNode.
       boolean stale = !slot.isValid();
-      LOG.trace("{}: checked shared memory segment.  isStale={}", this, stale);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": checked shared memory segment.  isStale=" + stale);
+      }
       return stale;
     } else {
       // Fall back to old, time-based staleness method.
       long deltaMs = Time.monotonicNow() - creationTimeMs;
       long staleThresholdMs = cache.getStaleThresholdMs();
       if (deltaMs > staleThresholdMs) {
-        LOG.trace("{} is stale because it's {} ms old and staleThreadholdMS={}",
-            this, deltaMs, staleThresholdMs);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + " is stale because it's " + deltaMs +
+              " ms old, and staleThresholdMs = " + staleThresholdMs);
+        }
         return true;
       } else {
-        LOG.trace("{} is not stale because it's only {} ms old "
-            + "and staleThresholdMs={}",  this, deltaMs, staleThresholdMs);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(this + " is not stale because it's only " + deltaMs +
+              " ms old, and staleThresholdMs = " + staleThresholdMs);
+        }
         return false;
       }
     }
@@ -188,8 +194,13 @@ public class ShortCircuitReplica {
       return false;
     }
     boolean result = slot.addAnchor();
-    LOG.trace("{}: {} no-checksum anchor to slot {}",
-        this, result ? "added" : "could not add", slot);
+    if (LOG.isTraceEnabled()) {
+      if (result) {
+        LOG.trace(this + ": added no-checksum anchor to slot " + slot);
+      } else {
+        LOG.trace(this + ": could not add no-checksum anchor to slot " + slot);
+      }
+    }
     return result;
   }
 
@@ -252,7 +263,9 @@ public class ShortCircuitReplica {
         suffix += "  scheduling " + slot + " for later release.";
       }
     }
-    LOG.trace("closed {}{}", this, suffix);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("closed " + this + suffix);
+    }
   }
 
   public FileInputStream getDataStream() {
@@ -280,7 +293,9 @@ public class ShortCircuitReplica {
       FileChannel channel = dataStream.getChannel();
       MappedByteBuffer mmap = channel.map(MapMode.READ_ONLY, 0, 
           Math.min(Integer.MAX_VALUE, channel.size()));
-      LOG.trace("{}: created mmap of size {}", this, channel.size());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(this + ": created mmap of size " + channel.size());
+      }
       return mmap;
     } catch (IOException e) {
       LOG.warn(this + ": mmap error", e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
index fa40c15..78325a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
@@ -484,9 +484,13 @@ public class ShortCircuitShm {
         POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, mmappedLength);
     this.slots = new Slot[mmappedLength / BYTES_PER_SLOT];
     this.allocatedSlots = new BitSet(slots.length);
-    LOG.trace("creating {}(shmId={}, mmappedLength={}, baseAddress={}, "
-        + "slots.length={})", this.getClass().getSimpleName(), shmId,
-        mmappedLength, String.format("%x", baseAddress), slots.length);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("creating " + this.getClass().getSimpleName() +
+          "(shmId=" + shmId +
+          ", mmappedLength=" + mmappedLength +
+          ", baseAddress=" + String.format("%x", baseAddress) +
+          ", slots.length=" + slots.length + ")");
+    }
   }
 
   public final ShmId getShmId() {
@@ -611,7 +615,9 @@ public class ShortCircuitShm {
         "tried to unregister slot " + slotIdx + ", which was not registered.");
     allocatedSlots.set(slotIdx, false);
     slots[slotIdx] = null;
-    LOG.trace("{}: unregisterSlot {}", this, slotIdx);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + ": unregisterSlot " + slotIdx);
+    }
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
index e361252..a9adb7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
@@ -36,6 +36,18 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public abstract class ByteArrayManager {
   static final Logger LOG = LoggerFactory.getLogger(ByteArrayManager.class);
+  private static final ThreadLocal<StringBuilder> DEBUG_MESSAGE =
+      new ThreadLocal<StringBuilder>() {
+    protected StringBuilder initialValue() {
+      return new StringBuilder();
+    }
+  };
+
+  private static void logDebugMessage() {
+    final StringBuilder b = DEBUG_MESSAGE.get();
+    LOG.debug(b.toString());
+    b.setLength(0);
+  }
 
   static final int MIN_ARRAY_LENGTH = 32;
   static final byte[] EMPTY_BYTE_ARRAY = {};
@@ -148,18 +160,27 @@ public abstract class ByteArrayManager {
      * via the {@link FixedLengthManager#recycle(byte[])} method.
      */
     synchronized byte[] allocate() throws InterruptedException {
-      LOG.debug(", {}", this);
+      if (LOG.isDebugEnabled()) {
+        DEBUG_MESSAGE.get().append(", ").append(this);
+      }
       for(; numAllocated >= maxAllocated;) {
-        LOG.debug(": wait ...");
+        if (LOG.isDebugEnabled()) {
+          DEBUG_MESSAGE.get().append(": wait ...");
+          logDebugMessage();
+        }
 
         wait();
 
-        LOG.debug("wake up: {}", this);
+        if (LOG.isDebugEnabled()) {
+          DEBUG_MESSAGE.get().append("wake up: ").append(this);
+        }
       }
       numAllocated++;
 
       final byte[] array = freeQueue.poll();
-      LOG.debug(", recycled? {}", array != null);
+      if (LOG.isDebugEnabled()) {
+        DEBUG_MESSAGE.get().append(", recycled? ").append(array != null);
+      }
       return array != null? array : new byte[byteArrayLength];
     }
 
@@ -173,7 +194,9 @@ public abstract class ByteArrayManager {
     synchronized int recycle(byte[] array) {
       Preconditions.checkNotNull(array);
       Preconditions.checkArgument(array.length == byteArrayLength);
-      LOG.debug(", {}", this);
+      if (LOG.isDebugEnabled()) {
+        DEBUG_MESSAGE.get().append(", ").append(this);
+      }
 
       notify();
       numAllocated--;
@@ -184,7 +207,9 @@ public abstract class ByteArrayManager {
       }
 
       if (freeQueue.size() < maxAllocated - numAllocated) {
-        LOG.debug(", freeQueue.offer");
+        if (LOG.isDebugEnabled()) {
+          DEBUG_MESSAGE.get().append(", freeQueue.offer");
+        }
         freeQueue.offer(array);
       }
       return freeQueue.size();
@@ -324,7 +349,9 @@ public abstract class ByteArrayManager {
     public byte[] newByteArray(final int arrayLength)
         throws InterruptedException {
       Preconditions.checkArgument(arrayLength >= 0);
-      LOG.debug("allocate({})", arrayLength);
+      if (LOG.isDebugEnabled()) {
+        DEBUG_MESSAGE.get().append("allocate(").append(arrayLength).append(")");
+      }
 
       final byte[] array;
       if (arrayLength == 0) {
@@ -338,12 +365,18 @@ public abstract class ByteArrayManager {
         final FixedLengthManager manager =
             managers.get(powerOfTwo, aboveThreshold);
 
-        LOG.debug(": count={}, {}Threshold", count,
-            aboveThreshold ? "above" : "below");
+        if (LOG.isDebugEnabled()) {
+          DEBUG_MESSAGE.get().append(": count=").append(count)
+              .append(aboveThreshold? ", aboveThreshold": ", belowThreshold");
+        }
         array = manager != null? manager.allocate(): new byte[powerOfTwo];
       }
 
-      LOG.debug(", return byte[{}]", array.length);
+      if (LOG.isDebugEnabled()) {
+        DEBUG_MESSAGE.get().append(", return byte[")
+            .append(array.length).append("]");
+        logDebugMessage();
+      }
       return array;
     }
 
@@ -358,7 +391,10 @@ public abstract class ByteArrayManager {
     @Override
     public int release(final byte[] array) {
       Preconditions.checkNotNull(array);
-      LOG.debug("recycle: array.length={}", array.length);
+      if (LOG.isDebugEnabled()) {
+        DEBUG_MESSAGE.get()
+            .append("recycle: array.length=").append(array.length);
+      }
 
       final int freeQueueSize;
       if (array.length == 0) {
@@ -368,7 +404,10 @@ public abstract class ByteArrayManager {
         freeQueueSize = manager == null? -1: manager.recycle(array);
       }
 
-      LOG.debug(", freeQueueSize={}", freeQueueSize);
+      if (LOG.isDebugEnabled()) {
+        DEBUG_MESSAGE.get().append(", freeQueueSize=").append(freeQueueSize);
+        logDebugMessage();
+      }
       return freeQueueSize;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
index 870103e..a864d37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
@@ -134,7 +134,9 @@ final class TokenAspect<T extends FileSystem & Renewable> {
       if (token != null) {
         fs.setDelegationToken(token);
         addRenewAction(fs);
-        LOG.debug("Created new DT for {}", token.getService());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Created new DT for {}", token.getService());
+        }
       }
       hasInitedToken = true;
     }
@@ -147,7 +149,9 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   synchronized void initDelegationToken(UserGroupInformation ugi) {
     Token<?> token = selectDelegationToken(ugi);
     if (token != null) {
-      LOG.debug("Found existing DT for {}", token.getService());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found existing DT for {}", token.getService());
+      }
       fs.setDelegationToken(token);
       hasInitedToken = true;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ee0539e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
index be5f17d..4c23241 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
@@ -182,7 +182,9 @@ public class URLConnectionFactory {
   public URLConnection openConnection(URL url, boolean isSpnego)
       throws IOException, AuthenticationException {
     if (isSpnego) {
-      LOG.debug("open AuthenticatedURL connection {}", url);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("open AuthenticatedURL connection {}", url);
+      }
       UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
       final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
       return new AuthenticatedURL(new KerberosUgiAuthenticator(),


[37/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index 0000000,0000000..69105a0
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@@ -1,0 -1,0 +1,972 @@@
++/**
++ * 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.base.Preconditions;
++import org.apache.hadoop.fs.ChecksumException;
++import org.apache.hadoop.fs.ReadOption;
++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.LocatedStripedBlock;
++import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
++import org.apache.hadoop.hdfs.util.StripedBlockUtil;
++import org.apache.hadoop.io.ByteBufferPool;
++
++import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
++import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
++import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
++
++import org.apache.hadoop.io.IOUtils;
++import org.apache.hadoop.io.erasurecode.CodecUtil;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
++
++import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
++import org.apache.hadoop.util.DirectBufferPool;
++
++import java.io.EOFException;
++import java.io.IOException;
++import java.io.InterruptedIOException;
++import java.nio.ByteBuffer;
++import java.util.ArrayList;
++import java.util.Arrays;
++import java.util.Collections;
++import java.util.EnumSet;
++import java.util.List;
++import java.util.Set;
++import java.util.Collection;
++import java.util.Map;
++import java.util.HashMap;
++import java.util.concurrent.CompletionService;
++import java.util.concurrent.ConcurrentHashMap;
++import java.util.concurrent.ExecutorCompletionService;
++import java.util.concurrent.Callable;
++import java.util.concurrent.Future;
++
++/**
++ * DFSStripedInputStream reads from striped block groups
++ */
++public class DFSStripedInputStream extends DFSInputStream {
++
++  private static class ReaderRetryPolicy {
++    private int fetchEncryptionKeyTimes = 1;
++    private int fetchTokenTimes = 1;
++
++    void refetchEncryptionKey() {
++      fetchEncryptionKeyTimes--;
++    }
++
++    void refetchToken() {
++      fetchTokenTimes--;
++    }
++
++    boolean shouldRefetchEncryptionKey() {
++      return fetchEncryptionKeyTimes > 0;
++    }
++
++    boolean shouldRefetchToken() {
++      return fetchTokenTimes > 0;
++    }
++  }
++
++  /** Used to indicate the buffered data's range in the block group */
++  private static class StripeRange {
++    /** start offset in the block group (inclusive) */
++    final long offsetInBlock;
++    /** length of the stripe range */
++    final long length;
++
++    StripeRange(long offsetInBlock, long length) {
++      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
++      this.offsetInBlock = offsetInBlock;
++      this.length = length;
++    }
++
++    boolean include(long pos) {
++      return pos >= offsetInBlock && pos < offsetInBlock + length;
++    }
++  }
++
++  private static class BlockReaderInfo {
++    final BlockReader reader;
++    final DatanodeInfo datanode;
++    /**
++     * when initializing block readers, their starting offsets are set to the same
++     * number: the smallest internal block offsets among all the readers. This is
++     * because it is possible that for some internal blocks we have to read
++     * "backwards" for decoding purpose. We thus use this offset array to track
++     * offsets for all the block readers so that we can skip data if necessary.
++     */
++    long blockReaderOffset;
++    /**
++     * We use this field to indicate whether we should use this reader. In case
++     * we hit any issue with this reader, we set this field to true and avoid
++     * using it for the next stripe.
++     */
++    boolean shouldSkip = false;
++
++    BlockReaderInfo(BlockReader reader, DatanodeInfo dn, long offset) {
++      this.reader = reader;
++      this.datanode = dn;
++      this.blockReaderOffset = offset;
++    }
++
++    void setOffset(long offset) {
++      this.blockReaderOffset = offset;
++    }
++
++    void skip() {
++      this.shouldSkip = true;
++    }
++  }
++
++  private static final DirectBufferPool bufferPool = new DirectBufferPool();
++
++  private final BlockReaderInfo[] blockReaders;
++  private final int cellSize;
++  private final short dataBlkNum;
++  private final short parityBlkNum;
++  private final int groupSize;
++  /** the buffer for a complete stripe */
++  private ByteBuffer curStripeBuf;
++  private ByteBuffer parityBuf;
++  private final ErasureCodingPolicy ecPolicy;
++  private final RawErasureDecoder decoder;
++
++  /**
++   * indicate the start/end offset of the current buffered stripe in the
++   * block group
++   */
++  private StripeRange curStripeRange;
++  private final CompletionService<Void> readingService;
++
++  /**
++   * When warning the user of a lost block in striping mode, we remember the
++   * dead nodes we've logged. All other striping blocks on these nodes can be
++   * considered lost too, and we don't want to log a warning for each of them.
++   * This is to prevent the log from being too verbose. Refer to HDFS-8920.
++   *
++   * To minimize the overhead, we only store the datanodeUuid in this set
++   */
++  private final Set<String> warnedNodes = Collections.newSetFromMap(
++      new ConcurrentHashMap<String, Boolean>());
++
++  DFSStripedInputStream(DFSClient dfsClient, String src,
++      boolean verifyChecksum, ErasureCodingPolicy ecPolicy,
++      LocatedBlocks locatedBlocks) throws IOException {
++    super(dfsClient, src, verifyChecksum, locatedBlocks);
++
++    assert ecPolicy != null;
++    this.ecPolicy = ecPolicy;
++    this.cellSize = ecPolicy.getCellSize();
++    dataBlkNum = (short) ecPolicy.getNumDataUnits();
++    parityBlkNum = (short) ecPolicy.getNumParityUnits();
++    groupSize = dataBlkNum + parityBlkNum;
++    blockReaders = new BlockReaderInfo[groupSize];
++    curStripeRange = new StripeRange(0, 0);
++    readingService =
++        new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
++    decoder = CodecUtil.createRSRawDecoder(dfsClient.getConfiguration(),
++        dataBlkNum, parityBlkNum);
++    if (DFSClient.LOG.isDebugEnabled()) {
++      DFSClient.LOG.debug("Creating an striped input stream for file " + src);
++    }
++  }
++
++  private void resetCurStripeBuffer() {
++    if (curStripeBuf == null) {
++      curStripeBuf = bufferPool.getBuffer(cellSize * dataBlkNum);
++    }
++    curStripeBuf.clear();
++    curStripeRange = new StripeRange(0, 0);
++  }
++
++  private ByteBuffer getParityBuffer() {
++    if (parityBuf == null) {
++      parityBuf = bufferPool.getBuffer(cellSize * parityBlkNum);
++    }
++    parityBuf.clear();
++    return parityBuf;
++  }
++
++  /**
++   * When seeking into a new block group, create blockReader for each internal
++   * block in the group.
++   */
++  private synchronized void 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();
++
++    // Compute desired striped block group
++    LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target);
++    // Update current position
++    this.pos = target;
++    this.blockEnd = targetBlockGroup.getStartOffset() +
++        targetBlockGroup.getBlockSize() - 1;
++    currentLocatedBlock = targetBlockGroup;
++  }
++
++  @Override
++  public synchronized void close() throws IOException {
++    super.close();
++    if (curStripeBuf != null) {
++      bufferPool.returnBuffer(curStripeBuf);
++      curStripeBuf = null;
++    }
++    if (parityBuf != null) {
++      bufferPool.returnBuffer(parityBuf);
++      parityBuf = null;
++    }
++  }
++
++  /**
++   * Extend the super method with the logic of switching between cells.
++   * When reaching the end of a cell, proceed to the next cell and read it
++   * with the next blockReader.
++   */
++  @Override
++  protected void closeCurrentBlockReaders() {
++    resetCurStripeBuffer();
++    if (blockReaders ==  null || blockReaders.length == 0) {
++      return;
++    }
++    for (int i = 0; i < groupSize; i++) {
++      closeReader(blockReaders[i]);
++      blockReaders[i] = null;
++    }
++    blockEnd = -1;
++  }
++
++  private void closeReader(BlockReaderInfo readerInfo) {
++    if (readerInfo != null) {
++//      IOUtils.cleanup(null, readerInfo.reader);
++      readerInfo.skip();
++    }
++  }
++
++  private long getOffsetInBlockGroup() {
++    return getOffsetInBlockGroup(pos);
++  }
++
++  private long getOffsetInBlockGroup(long pos) {
++    return pos - currentLocatedBlock.getStartOffset();
++  }
++
++  /**
++   * Read a new stripe covering the current position, and store the data in the
++   * {@link #curStripeBuf}.
++   */
++  private void readOneStripe(
++      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
++      throws IOException {
++    resetCurStripeBuffer();
++
++    // compute stripe range based on pos
++    final long offsetInBlockGroup = getOffsetInBlockGroup();
++    final long stripeLen = cellSize * dataBlkNum;
++    final int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
++    final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen);
++    final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize()
++        - (stripeIndex * stripeLen), stripeLen);
++    StripeRange stripeRange = new StripeRange(offsetInBlockGroup,
++        stripeLimit - stripeBufOffset);
++
++    LocatedStripedBlock blockGroup = (LocatedStripedBlock) currentLocatedBlock;
++    AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(ecPolicy, cellSize,
++        blockGroup, offsetInBlockGroup,
++        offsetInBlockGroup + stripeRange.length - 1, curStripeBuf);
++    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
++        blockGroup, cellSize, dataBlkNum, parityBlkNum);
++    // read the whole stripe
++    for (AlignedStripe stripe : stripes) {
++      // Parse group to get chosen DN location
++      StripeReader sreader = new StatefulStripeReader(readingService, stripe,
++          blks, blockReaders, corruptedBlockMap);
++      sreader.readStripe();
++    }
++    curStripeBuf.position(stripeBufOffset);
++    curStripeBuf.limit(stripeLimit);
++    curStripeRange = stripeRange;
++  }
++
++  private Callable<Void> readCells(final BlockReader reader,
++      final DatanodeInfo datanode, final long currentReaderOffset,
++      final long targetReaderOffset, final ByteBufferStrategy[] strategies,
++      final ExtendedBlock currentBlock,
++      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++    return new Callable<Void>() {
++      @Override
++      public Void call() throws Exception {
++        // reader can be null if getBlockReaderWithRetry failed or
++        // the reader hit exception before
++        if (reader == null) {
++          throw new IOException("The BlockReader is null. " +
++              "The BlockReader creation failed or the reader hit exception.");
++        }
++        Preconditions.checkState(currentReaderOffset <= targetReaderOffset);
++        if (currentReaderOffset < targetReaderOffset) {
++          long skipped = reader.skip(targetReaderOffset - currentReaderOffset);
++          Preconditions.checkState(
++              skipped == targetReaderOffset - currentReaderOffset);
++        }
++        int result = 0;
++        for (ByteBufferStrategy strategy : strategies) {
++          result += readToBuffer(reader, datanode, strategy, currentBlock,
++              corruptedBlockMap);
++        }
++        return null;
++      }
++    };
++  }
++
++  private int readToBuffer(BlockReader blockReader,
++      DatanodeInfo currentNode, ByteBufferStrategy strategy,
++      ExtendedBlock currentBlock,
++      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
++      throws IOException {
++    final int targetLength = strategy.buf.remaining();
++    int length = 0;
++    try {
++      while (length < targetLength) {
++        int ret = strategy.doRead(blockReader, 0, 0);
++        if (ret < 0) {
++          throw new IOException("Unexpected EOS from the reader");
++        }
++        length += ret;
++      }
++      return length;
++    } catch (ChecksumException ce) {
++      DFSClient.LOG.warn("Found Checksum error for "
++          + currentBlock + " from " + currentNode
++          + " at " + ce.getPos());
++      // we want to remember which block replicas we have tried
++      addIntoCorruptedBlockMap(currentBlock, currentNode,
++          corruptedBlockMap);
++      throw ce;
++    } catch (IOException e) {
++      DFSClient.LOG.warn("Exception while reading from "
++          + currentBlock + " of " + src + " from "
++          + currentNode, e);
++      throw e;
++    }
++  }
++
++  /**
++   * 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!");
++    }
++    if (targetPos <= blockEnd) {
++      final long targetOffsetInBlk = getOffsetInBlockGroup(targetPos);
++      if (curStripeRange.include(targetOffsetInBlk)) {
++        int bufOffset = getStripedBufOffset(targetOffsetInBlk);
++        curStripeBuf.position(bufOffset);
++        pos = targetPos;
++        return;
++      }
++    }
++    pos = targetPos;
++    blockEnd = -1;
++  }
++
++  private int getStripedBufOffset(long offsetInBlockGroup) {
++    final long stripeLen = cellSize * dataBlkNum;
++    // compute the position in the curStripeBuf based on "pos"
++    return (int) (offsetInBlockGroup % stripeLen);
++  }
++
++  @Override
++  public synchronized boolean seekToNewSource(long targetPos)
++      throws IOException {
++    return false;
++  }
++
++  @Override
++  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 ConcurrentHashMap<>();
++    if (pos < getFileLength()) {
++      try {
++        if (pos > blockEnd) {
++          blockSeekTo(pos);
++        }
++        int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
++        synchronized (infoLock) {
++          if (locatedBlocks.isLastBlockComplete()) {
++            realLen = (int) Math.min(realLen,
++                locatedBlocks.getFileLength() - pos);
++          }
++        }
++
++        /** Number of bytes already read into buffer */
++        int result = 0;
++        while (result < realLen) {
++          if (!curStripeRange.include(getOffsetInBlockGroup())) {
++            readOneStripe(corruptedBlockMap);
++          }
++          int ret = copyToTargetBuf(strategy, off + result, realLen - result);
++          result += ret;
++          pos += ret;
++        }
++        if (dfsClient.stats != null) {
++          dfsClient.stats.incrementBytesRead(result);
++        }
++        return result;
++      } finally {
++        // Check if need to report block replicas corruption either read
++        // was successful or ChecksumException occured.
++        reportCheckSumFailure(corruptedBlockMap,
++            currentLocatedBlock.getLocations().length);
++      }
++    }
++    return -1;
++  }
++
++  /**
++   * Copy the data from {@link #curStripeBuf} into the given buffer
++   * @param strategy the ReaderStrategy containing the given buffer
++   * @param offset the offset of the given buffer. Used only when strategy is
++   *               a ByteArrayStrategy
++   * @param length target length
++   * @return number of bytes copied
++   */
++  private int copyToTargetBuf(ReaderStrategy strategy, int offset, int length) {
++    final long offsetInBlk = getOffsetInBlockGroup();
++    int bufOffset = getStripedBufOffset(offsetInBlk);
++    curStripeBuf.position(bufOffset);
++    return strategy.copyFrom(curStripeBuf, offset,
++        Math.min(length, curStripeBuf.remaining()));
++  }
++
++  /**
++   * The super method {@link DFSInputStream#refreshLocatedBlock} refreshes
++   * cached LocatedBlock by executing {@link DFSInputStream#getBlockAt} again.
++   * This method extends the logic by first remembering the index of the
++   * internal block, and re-parsing the refreshed block group with the same
++   * index.
++   */
++  @Override
++  protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
++      throws IOException {
++    int idx = StripedBlockUtil.getBlockIndex(block.getBlock().getLocalBlock());
++    LocatedBlock lb = getBlockGroupAt(block.getStartOffset());
++    // If indexing information is returned, iterate through the index array
++    // to find the entry for position idx in the group
++    LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
++    int i = 0;
++    for (; i < lsb.getBlockIndices().length; i++) {
++      if (lsb.getBlockIndices()[i] == idx) {
++        break;
++      }
++    }
++    if (DFSClient.LOG.isDebugEnabled()) {
++      DFSClient.LOG.debug("refreshLocatedBlock for striped blocks, offset="
++          + block.getStartOffset() + ". Obtained block " + lb + ", idx=" + idx);
++    }
++    return StripedBlockUtil.constructInternalBlock(
++        lsb, i, cellSize, dataBlkNum, idx);
++  }
++
++  private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException {
++    LocatedBlock lb = super.getBlockAt(offset);
++    assert lb instanceof LocatedStripedBlock : "NameNode" +
++        " should return a LocatedStripedBlock for a striped file";
++    return (LocatedStripedBlock)lb;
++  }
++
++  /**
++   * Real implementation of pread.
++   */
++  @Override
++  protected void fetchBlockByteRange(LocatedBlock block, long start,
++      long end, byte[] buf, int offset,
++      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
++      throws IOException {
++    // Refresh the striped block group
++    LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset());
++
++    AlignedStripe[] stripes = StripedBlockUtil.divideByteRangeIntoStripes(
++        ecPolicy, cellSize, blockGroup, start, end, buf, offset);
++    CompletionService<Void> readService = new ExecutorCompletionService<>(
++        dfsClient.getStripedReadsThreadPool());
++    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
++        blockGroup, cellSize, dataBlkNum, parityBlkNum);
++    final BlockReaderInfo[] preaderInfos = new BlockReaderInfo[groupSize];
++    try {
++      for (AlignedStripe stripe : stripes) {
++        // Parse group to get chosen DN location
++        StripeReader preader = new PositionStripeReader(readService, stripe,
++            blks, preaderInfos, corruptedBlockMap);
++        preader.readStripe();
++      }
++    } finally {
++      for (BlockReaderInfo preaderInfo : preaderInfos) {
++        closeReader(preaderInfo);
++      }
++    }
++  }
++
++  @Override
++  protected void reportLostBlock(LocatedBlock lostBlock,
++      Collection<DatanodeInfo> ignoredNodes) {
++    DatanodeInfo[] nodes = lostBlock.getLocations();
++    if (nodes != null && nodes.length > 0) {
++      List<String> dnUUIDs = new ArrayList<>();
++      for (DatanodeInfo node : nodes) {
++        dnUUIDs.add(node.getDatanodeUuid());
++      }
++      if (!warnedNodes.containsAll(dnUUIDs)) {
++        DFSClient.LOG.warn(Arrays.toString(nodes) + " are unavailable and " +
++            "all striping blocks on them are lost. " +
++            "IgnoredNodes = " + ignoredNodes);
++        warnedNodes.addAll(dnUUIDs);
++      }
++    } else {
++      super.reportLostBlock(lostBlock, ignoredNodes);
++    }
++  }
++
++  /**
++   * The reader for reading a complete {@link AlignedStripe}. Note that an
++   * {@link AlignedStripe} may cross multiple stripes with cellSize width.
++   */
++  private abstract class StripeReader {
++    final Map<Future<Void>, Integer> futures = new HashMap<>();
++    final AlignedStripe alignedStripe;
++    final CompletionService<Void> service;
++    final LocatedBlock[] targetBlocks;
++    final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap;
++    final BlockReaderInfo[] readerInfos;
++
++    StripeReader(CompletionService<Void> service, AlignedStripe alignedStripe,
++        LocatedBlock[] targetBlocks, BlockReaderInfo[] readerInfos,
++        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++      this.service = service;
++      this.alignedStripe = alignedStripe;
++      this.targetBlocks = targetBlocks;
++      this.readerInfos = readerInfos;
++      this.corruptedBlockMap = corruptedBlockMap;
++    }
++
++    /** prepare all the data chunks */
++    abstract void prepareDecodeInputs();
++
++    /** prepare the parity chunk and block reader if necessary */
++    abstract boolean prepareParityChunk(int index) throws IOException;
++
++    abstract void decode();
++
++    void updateState4SuccessRead(StripingChunkReadResult result) {
++      Preconditions.checkArgument(
++          result.state == StripingChunkReadResult.SUCCESSFUL);
++      readerInfos[result.index].setOffset(alignedStripe.getOffsetInBlock()
++          + alignedStripe.getSpanInBlock());
++    }
++
++    private void checkMissingBlocks() throws IOException {
++      if (alignedStripe.missingChunksNum > parityBlkNum) {
++        clearFutures(futures.keySet());
++        throw new IOException(alignedStripe.missingChunksNum
++            + " missing blocks, the stripe is: " + alignedStripe);
++      }
++    }
++
++    /**
++     * We need decoding. Thus go through all the data chunks and make sure we
++     * submit read requests for all of them.
++     */
++    private void readDataForDecoding() throws IOException {
++      prepareDecodeInputs();
++      for (int i = 0; i < dataBlkNum; i++) {
++        Preconditions.checkNotNull(alignedStripe.chunks[i]);
++        if (alignedStripe.chunks[i].state == StripingChunk.REQUESTED) {
++          if (!readChunk(targetBlocks[i], i)) {
++            alignedStripe.missingChunksNum++;
++          }
++        }
++      }
++      checkMissingBlocks();
++    }
++
++    void readParityChunks(int num) throws IOException {
++      for (int i = dataBlkNum, j = 0; i < dataBlkNum + parityBlkNum && j < num;
++           i++) {
++        if (alignedStripe.chunks[i] == null) {
++          if (prepareParityChunk(i) && readChunk(targetBlocks[i], i)) {
++            j++;
++          } else {
++            alignedStripe.missingChunksNum++;
++          }
++        }
++      }
++      checkMissingBlocks();
++    }
++
++    boolean createBlockReader(LocatedBlock block, int chunkIndex)
++        throws IOException {
++      BlockReader reader = null;
++      final ReaderRetryPolicy retry = new ReaderRetryPolicy();
++      DNAddrPair dnInfo = new DNAddrPair(null, null, null);
++
++      while(true) {
++        try {
++          // the cached block location might have been re-fetched, so always
++          // get it from cache.
++          block = refreshLocatedBlock(block);
++          targetBlocks[chunkIndex] = block;
++
++          // internal block has one location, just rule out the deadNodes
++          dnInfo = getBestNodeDNAddrPair(block, null);
++          if (dnInfo == null) {
++            break;
++          }
++          reader = getBlockReader(block, alignedStripe.getOffsetInBlock(),
++              block.getBlockSize() - alignedStripe.getOffsetInBlock(),
++              dnInfo.addr, dnInfo.storageType, dnInfo.info);
++        } catch (IOException e) {
++          if (e instanceof InvalidEncryptionKeyException &&
++              retry.shouldRefetchEncryptionKey()) {
++            DFSClient.LOG.info("Will fetch a new encryption key and retry, "
++                + "encryption key was invalid when connecting to " + dnInfo.addr
++                + " : " + e);
++            dfsClient.clearDataEncryptionKey();
++            retry.refetchEncryptionKey();
++          } else if (retry.shouldRefetchToken() &&
++              tokenRefetchNeeded(e, dnInfo.addr)) {
++            fetchBlockAt(block.getStartOffset());
++            retry.refetchToken();
++          } else {
++            //TODO: handles connection issues
++            DFSClient.LOG.warn("Failed to connect to " + dnInfo.addr + " for " +
++                "block" + block.getBlock(), e);
++            // re-fetch the block in case the block has been moved
++            fetchBlockAt(block.getStartOffset());
++            addToDeadNodes(dnInfo.info);
++          }
++        }
++        if (reader != null) {
++          readerInfos[chunkIndex] = new BlockReaderInfo(reader, dnInfo.info,
++              alignedStripe.getOffsetInBlock());
++          return true;
++        }
++      }
++      return false;
++    }
++
++    private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) {
++      if (chunk.byteBuffer != null) {
++        ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer);
++        return new ByteBufferStrategy[]{strategy};
++      } else {
++        ByteBufferStrategy[] strategies =
++            new ByteBufferStrategy[chunk.byteArray.getOffsets().length];
++        for (int i = 0; i < strategies.length; i++) {
++          ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(),
++              chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]);
++          strategies[i] = new ByteBufferStrategy(buffer);
++        }
++        return strategies;
++      }
++    }
++
++    boolean readChunk(final LocatedBlock block, int chunkIndex)
++        throws IOException {
++      final StripingChunk chunk = alignedStripe.chunks[chunkIndex];
++      if (block == null) {
++        chunk.state = StripingChunk.MISSING;
++        return false;
++      }
++      if (readerInfos[chunkIndex] == null) {
++        if (!createBlockReader(block, chunkIndex)) {
++          chunk.state = StripingChunk.MISSING;
++          return false;
++        }
++      } else if (readerInfos[chunkIndex].shouldSkip) {
++        chunk.state = StripingChunk.MISSING;
++        return false;
++      }
++
++      chunk.state = StripingChunk.PENDING;
++      Callable<Void> readCallable = readCells(readerInfos[chunkIndex].reader,
++          readerInfos[chunkIndex].datanode,
++          readerInfos[chunkIndex].blockReaderOffset,
++          alignedStripe.getOffsetInBlock(), getReadStrategies(chunk),
++          block.getBlock(), corruptedBlockMap);
++
++      Future<Void> request = service.submit(readCallable);
++      futures.put(request, chunkIndex);
++      return true;
++    }
++
++    /** read the whole stripe. do decoding if necessary */
++    void readStripe() throws IOException {
++      for (int i = 0; i < dataBlkNum; i++) {
++        if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state != StripingChunk.ALLZERO) {
++          if (!readChunk(targetBlocks[i], i)) {
++            alignedStripe.missingChunksNum++;
++          }
++        }
++      }
++      // There are missing block locations at this stage. Thus we need to read
++      // the full stripe and one more parity block.
++      if (alignedStripe.missingChunksNum > 0) {
++        checkMissingBlocks();
++        readDataForDecoding();
++        // read parity chunks
++        readParityChunks(alignedStripe.missingChunksNum);
++      }
++      // TODO: for a full stripe we can start reading (dataBlkNum + 1) chunks
++
++      // Input buffers for potential decode operation, which remains null until
++      // first read failure
++      while (!futures.isEmpty()) {
++        try {
++          StripingChunkReadResult r = StripedBlockUtil
++              .getNextCompletedStripedRead(service, futures, 0);
++          if (DFSClient.LOG.isDebugEnabled()) {
++            DFSClient.LOG.debug("Read task returned: " + r + ", for stripe "
++                + alignedStripe);
++          }
++          StripingChunk returnedChunk = alignedStripe.chunks[r.index];
++          Preconditions.checkNotNull(returnedChunk);
++          Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING);
++
++          if (r.state == StripingChunkReadResult.SUCCESSFUL) {
++            returnedChunk.state = StripingChunk.FETCHED;
++            alignedStripe.fetchedChunksNum++;
++            updateState4SuccessRead(r);
++            if (alignedStripe.fetchedChunksNum == dataBlkNum) {
++              clearFutures(futures.keySet());
++              break;
++            }
++          } else {
++            returnedChunk.state = StripingChunk.MISSING;
++            // close the corresponding reader
++            closeReader(readerInfos[r.index]);
++
++            final int missing = alignedStripe.missingChunksNum;
++            alignedStripe.missingChunksNum++;
++            checkMissingBlocks();
++
++            readDataForDecoding();
++            readParityChunks(alignedStripe.missingChunksNum - missing);
++          }
++        } catch (InterruptedException ie) {
++          String err = "Read request interrupted";
++          DFSClient.LOG.error(err);
++          clearFutures(futures.keySet());
++          // Don't decode if read interrupted
++          throw new InterruptedIOException(err);
++        }
++      }
++
++      if (alignedStripe.missingChunksNum > 0) {
++        decode();
++      }
++    }
++  }
++
++  class PositionStripeReader extends StripeReader {
++    private byte[][] decodeInputs = null;
++
++    PositionStripeReader(CompletionService<Void> service,
++        AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
++        BlockReaderInfo[] readerInfos,
++        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++      super(service, alignedStripe, targetBlocks, readerInfos,
++          corruptedBlockMap);
++    }
++
++    @Override
++    void prepareDecodeInputs() {
++      if (decodeInputs == null) {
++        decodeInputs = StripedBlockUtil.initDecodeInputs(alignedStripe,
++            dataBlkNum, parityBlkNum);
++      }
++    }
++
++    @Override
++    boolean prepareParityChunk(int index) {
++      Preconditions.checkState(index >= dataBlkNum &&
++          alignedStripe.chunks[index] == null);
++      final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index,
++          dataBlkNum, parityBlkNum);
++      alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
++      alignedStripe.chunks[index].addByteArraySlice(0,
++          (int) alignedStripe.getSpanInBlock());
++      return true;
++    }
++
++    @Override
++    void decode() {
++      StripedBlockUtil.finalizeDecodeInputs(decodeInputs, dataBlkNum,
++          parityBlkNum, alignedStripe);
++      StripedBlockUtil.decodeAndFillBuffer(decodeInputs, alignedStripe,
++          dataBlkNum, parityBlkNum, decoder);
++    }
++  }
++
++  class StatefulStripeReader extends StripeReader {
++    ByteBuffer[] decodeInputs;
++
++    StatefulStripeReader(CompletionService<Void> service,
++        AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
++        BlockReaderInfo[] readerInfos,
++        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++      super(service, alignedStripe, targetBlocks, readerInfos,
++          corruptedBlockMap);
++    }
++
++    @Override
++    void prepareDecodeInputs() {
++      if (decodeInputs == null) {
++        decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum];
++        final ByteBuffer cur;
++        synchronized (DFSStripedInputStream.this) {
++          cur = curStripeBuf.duplicate();
++        }
++        StripedBlockUtil.VerticalRange range = alignedStripe.range;
++        for (int i = 0; i < dataBlkNum; i++) {
++          cur.limit(cur.capacity());
++          int pos = (int) (range.offsetInBlock % cellSize + cellSize * i);
++          cur.position(pos);
++          cur.limit((int) (pos + range.spanInBlock));
++          final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
++              dataBlkNum, parityBlkNum);
++          decodeInputs[decodeIndex] = cur.slice();
++          if (alignedStripe.chunks[i] == null) {
++            alignedStripe.chunks[i] = new StripingChunk(
++                decodeInputs[decodeIndex]);
++          }
++        }
++      }
++    }
++
++    @Override
++    boolean prepareParityChunk(int index) throws IOException {
++      Preconditions.checkState(index >= dataBlkNum
++          && alignedStripe.chunks[index] == null);
++      if (blockReaders[index] != null && blockReaders[index].shouldSkip) {
++        alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING);
++        // we have failed the block reader before
++        return false;
++      }
++      final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index,
++          dataBlkNum, parityBlkNum);
++      ByteBuffer buf = getParityBuffer().duplicate();
++      buf.position(cellSize * decodeIndex);
++      buf.limit(cellSize * decodeIndex + (int) alignedStripe.range.spanInBlock);
++      decodeInputs[decodeIndex] = buf.slice();
++      alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
++      return true;
++    }
++
++    @Override
++    void decode() {
++      // TODO no copy for data chunks. this depends on HADOOP-12047
++      final int span = (int) alignedStripe.getSpanInBlock();
++      for (int i = 0; i < alignedStripe.chunks.length; i++) {
++        final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
++            dataBlkNum, parityBlkNum);
++        if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state == StripingChunk.ALLZERO) {
++          for (int j = 0; j < span; j++) {
++            decodeInputs[decodeIndex].put((byte) 0);
++          }
++          decodeInputs[decodeIndex].flip();
++        } else if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state == StripingChunk.FETCHED) {
++          decodeInputs[decodeIndex].position(0);
++          decodeInputs[decodeIndex].limit(span);
++        }
++      }
++      int[] decodeIndices = new int[parityBlkNum];
++      int pos = 0;
++      for (int i = 0; i < alignedStripe.chunks.length; i++) {
++        if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state == StripingChunk.MISSING) {
++          int  decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
++              dataBlkNum, parityBlkNum);
++          if (i < dataBlkNum) {
++            decodeIndices[pos++] = decodeIndex;
++          } else {
++            decodeInputs[decodeIndex] = null;
++          }
++        }
++      }
++      decodeIndices = Arrays.copyOf(decodeIndices, pos);
++
++      final int decodeChunkNum = decodeIndices.length;
++      ByteBuffer[] outputs = new ByteBuffer[decodeChunkNum];
++      for (int i = 0; i < decodeChunkNum; i++) {
++        outputs[i] = decodeInputs[decodeIndices[i]];
++        outputs[i].position(0);
++        outputs[i].limit((int) alignedStripe.range.spanInBlock);
++        decodeInputs[decodeIndices[i]] = null;
++      }
++
++      decoder.decode(decodeInputs, decodeIndices, outputs);
++    }
++  }
++
++  /**
++   * May need online read recovery, zero-copy read doesn't make
++   * sense, so don't support it.
++   */
++  @Override
++  public synchronized ByteBuffer read(ByteBufferPool bufferPool,
++      int maxLength, EnumSet<ReadOption> opts)
++          throws IOException, UnsupportedOperationException {
++    throw new UnsupportedOperationException(
++        "Not support enhanced byte buffer access.");
++  }
++
++  @Override
++  public synchronized void releaseBuffer(ByteBuffer buffer) {
++    throw new UnsupportedOperationException(
++        "Not support enhanced byte buffer access.");
++  }
++
++  /** A variation to {@link DFSInputStream#cancelAll} */
++  private void clearFutures(Collection<Future<Void>> futures) {
++    for (Future<Void> future : futures) {
++      future.cancel(false);
++    }
++    futures.clear();
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 0000000,0000000..bf4e10e
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@@ -1,0 -1,0 +1,953 @@@
++/**
++ * 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.io.InterruptedIOException;
++import java.nio.ByteBuffer;
++import java.nio.channels.ClosedChannelException;
++import java.util.ArrayList;
++import java.util.Arrays;
++import java.util.Collections;
++import java.util.EnumSet;
++import java.util.HashMap;
++import java.util.HashSet;
++import java.util.List;
++import java.util.Map;
++import java.util.Set;
++import java.util.concurrent.BlockingQueue;
++import java.util.concurrent.LinkedBlockingQueue;
++import java.util.concurrent.TimeUnit;
++
++import org.apache.hadoop.HadoopIllegalArgumentException;
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.fs.CreateFlag;
++import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
++import org.apache.hadoop.hdfs.protocol.ClientProtocol;
++import org.apache.hadoop.hdfs.protocol.DatanodeID;
++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.LocatedStripedBlock;
++import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
++import org.apache.hadoop.hdfs.util.StripedBlockUtil;
++import org.apache.hadoop.io.MultipleIOException;
++import org.apache.hadoop.io.erasurecode.CodecUtil;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
++import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
++import org.apache.hadoop.util.DataChecksum;
++import org.apache.hadoop.util.Progressable;
++import org.apache.hadoop.util.Time;
++
++import com.google.common.base.Preconditions;
++import org.apache.htrace.core.TraceScope;
++
++
++/**
++ * This class supports writing files in striped layout and erasure coded format.
++ * Each stripe contains a sequence of cells.
++ */
++@InterfaceAudience.Private
++public class DFSStripedOutputStream extends DFSOutputStream {
++  static class MultipleBlockingQueue<T> {
++    private final List<BlockingQueue<T>> queues;
++
++    MultipleBlockingQueue(int numQueue, int queueSize) {
++      List<BlockingQueue<T>> list = new ArrayList<>(numQueue);
++      for (int i = 0; i < numQueue; i++) {
++        list.add(new LinkedBlockingQueue<T>(queueSize));
++      }
++      queues = Collections.synchronizedList(list);
++    }
++
++    void offer(int i, T object) {
++      final boolean b = queues.get(i).offer(object);
++      Preconditions.checkState(b, "Failed to offer " + object
++          + " to queue, i=" + i);
++    }
++
++    T take(int i) throws InterruptedIOException {
++      try {
++        return queues.get(i).take();
++      } catch(InterruptedException ie) {
++        throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, ie);
++      }
++    }
++
++    T takeWithTimeout(int i) throws InterruptedIOException {
++      try {
++        return queues.get(i).poll(100, TimeUnit.MILLISECONDS);
++      } catch (InterruptedException e) {
++        throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, e);
++      }
++    }
++
++    T poll(int i) {
++      return queues.get(i).poll();
++    }
++
++    T peek(int i) {
++      return queues.get(i).peek();
++    }
++
++    void clear() {
++      for (BlockingQueue<T> q : queues) {
++        q.clear();
++      }
++    }
++  }
++
++  /** Coordinate the communication between the streamers. */
++  static class Coordinator {
++    /**
++     * The next internal block to write to for each streamers. The
++     * DFSStripedOutputStream makes the {@link ClientProtocol#addBlock} RPC to
++     * get a new block group. The block group is split to internal blocks, which
++     * are then distributed into the queue for streamers to retrieve.
++     */
++    private final MultipleBlockingQueue<LocatedBlock> followingBlocks;
++    /**
++     * Used to sync among all the streamers before allocating a new block. The
++     * DFSStripedOutputStream uses this to make sure every streamer has finished
++     * writing the previous block.
++     */
++    private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
++
++    /**
++     * The following data structures are used for syncing while handling errors
++     */
++    private final MultipleBlockingQueue<LocatedBlock> newBlocks;
++    private final Map<StripedDataStreamer, Boolean> updateStreamerMap;
++    private final MultipleBlockingQueue<Boolean> streamerUpdateResult;
++
++    Coordinator(final int numAllBlocks) {
++      followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
++      endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
++      newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
++      updateStreamerMap = Collections.synchronizedMap(
++          new HashMap<StripedDataStreamer, Boolean>(numAllBlocks));
++      streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1);
++    }
++
++    MultipleBlockingQueue<LocatedBlock> getFollowingBlocks() {
++      return followingBlocks;
++    }
++
++    MultipleBlockingQueue<LocatedBlock> getNewBlocks() {
++      return newBlocks;
++    }
++
++    void offerEndBlock(int i, ExtendedBlock block) {
++      endBlocks.offer(i, block);
++    }
++
++    void offerStreamerUpdateResult(int i, boolean success) {
++      streamerUpdateResult.offer(i, success);
++    }
++
++    boolean takeStreamerUpdateResult(int i) throws InterruptedIOException {
++      return streamerUpdateResult.take(i);
++    }
++
++    void updateStreamer(StripedDataStreamer streamer,
++        boolean success) {
++      assert !updateStreamerMap.containsKey(streamer);
++      updateStreamerMap.put(streamer, success);
++    }
++
++    void clearFailureStates() {
++      newBlocks.clear();
++      updateStreamerMap.clear();
++      streamerUpdateResult.clear();
++    }
++  }
++
++  /** Buffers for writing the data and parity cells of a stripe. */
++  class CellBuffers {
++    private final ByteBuffer[] buffers;
++    private final byte[][] checksumArrays;
++
++    CellBuffers(int numParityBlocks) throws InterruptedException{
++      if (cellSize % bytesPerChecksum != 0) {
++        throw new HadoopIllegalArgumentException("Invalid values: "
++            + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (="
++            + bytesPerChecksum + ") must divide cell size (=" + cellSize + ").");
++      }
++
++      checksumArrays = new byte[numParityBlocks][];
++      final int size = getChecksumSize() * (cellSize / bytesPerChecksum);
++      for (int i = 0; i < checksumArrays.length; i++) {
++        checksumArrays[i] = new byte[size];
++      }
++
++      buffers = new ByteBuffer[numAllBlocks];
++      for (int i = 0; i < buffers.length; i++) {
++        buffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
++      }
++    }
++
++    private ByteBuffer[] getBuffers() {
++      return buffers;
++    }
++
++    byte[] getChecksumArray(int i) {
++      return checksumArrays[i - numDataBlocks];
++    }
++
++    private int addTo(int i, byte[] b, int off, int len) {
++      final ByteBuffer buf = buffers[i];
++      final int pos = buf.position() + len;
++      Preconditions.checkState(pos <= cellSize);
++      buf.put(b, off, len);
++      return pos;
++    }
++
++    private void clear() {
++      for (int i = 0; i< numAllBlocks; i++) {
++        buffers[i].clear();
++        if (i >= numDataBlocks) {
++          Arrays.fill(buffers[i].array(), (byte) 0);
++        }
++      }
++    }
++
++    private void release() {
++      for (int i = 0; i < numAllBlocks; i++) {
++        byteArrayManager.release(buffers[i].array());
++      }
++    }
++
++    private void flipDataBuffers() {
++      for (int i = 0; i < numDataBlocks; i++) {
++        buffers[i].flip();
++      }
++    }
++  }
++
++  private final Coordinator coordinator;
++  private final CellBuffers cellBuffers;
++  private final RawErasureEncoder encoder;
++  private final List<StripedDataStreamer> streamers;
++  private final DFSPacket[] currentPackets; // current Packet of each streamer
++
++  /** Size of each striping cell, must be a multiple of bytesPerChecksum */
++  private final int cellSize;
++  private final int numAllBlocks;
++  private final int numDataBlocks;
++  private ExtendedBlock currentBlockGroup;
++  private final String[] favoredNodes;
++  private final List<StripedDataStreamer> failedStreamers;
++
++  /** Construct a new output stream for creating a file. */
++  DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
++                         EnumSet<CreateFlag> flag, Progressable progress,
++                         DataChecksum checksum, String[] favoredNodes)
++                         throws IOException {
++    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes, false);
++    if (LOG.isDebugEnabled()) {
++      LOG.debug("Creating DFSStripedOutputStream for " + src);
++    }
++
++    final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy();
++    final int numParityBlocks = ecPolicy.getNumParityUnits();
++    cellSize = ecPolicy.getCellSize();
++    numDataBlocks = ecPolicy.getNumDataUnits();
++    numAllBlocks = numDataBlocks + numParityBlocks;
++    this.favoredNodes = favoredNodes;
++    failedStreamers = new ArrayList<>();
++
++    encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
++        numDataBlocks, numParityBlocks);
++
++    coordinator = new Coordinator(numAllBlocks);
++    try {
++      cellBuffers = new CellBuffers(numParityBlocks);
++    } catch (InterruptedException ie) {
++      throw DFSUtilClient.toInterruptedIOException(
++          "Failed to create cell buffers", ie);
++    }
++
++    streamers = new ArrayList<>(numAllBlocks);
++    for (short i = 0; i < numAllBlocks; i++) {
++      StripedDataStreamer streamer = new StripedDataStreamer(stat,
++          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
++          favoredNodes, i, coordinator);
++      streamers.add(streamer);
++    }
++    currentPackets = new DFSPacket[streamers.size()];
++    setCurrentStreamer(0);
++  }
++
++  StripedDataStreamer getStripedDataStreamer(int i) {
++    return streamers.get(i);
++  }
++
++  int getCurrentIndex() {
++    return getCurrentStreamer().getIndex();
++  }
++
++  private synchronized StripedDataStreamer getCurrentStreamer() {
++    return (StripedDataStreamer) streamer;
++  }
++
++  private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) {
++    // backup currentPacket for current streamer
++    if (streamer != null) {
++      int oldIdx = streamers.indexOf(getCurrentStreamer());
++      if (oldIdx >= 0) {
++        currentPackets[oldIdx] = currentPacket;
++      }
++    }
++
++    streamer = getStripedDataStreamer(newIdx);
++    currentPacket = currentPackets[newIdx];
++    adjustChunkBoundary();
++
++    return getCurrentStreamer();
++  }
++
++  /**
++   * Encode the buffers, i.e. compute parities.
++   *
++   * @param buffers data buffers + parity buffers
++   */
++  private static void encode(RawErasureEncoder encoder, int numData,
++      ByteBuffer[] buffers) {
++    final ByteBuffer[] dataBuffers = new ByteBuffer[numData];
++    final ByteBuffer[] parityBuffers = new ByteBuffer[buffers.length - numData];
++    System.arraycopy(buffers, 0, dataBuffers, 0, dataBuffers.length);
++    System.arraycopy(buffers, numData, parityBuffers, 0, parityBuffers.length);
++
++    encoder.encode(dataBuffers, parityBuffers);
++  }
++
++  /**
++   * check all the existing StripedDataStreamer and find newly failed streamers.
++   * @return The newly failed streamers.
++   * @throws IOException if less than {@link #numDataBlocks} streamers are still
++   *                     healthy.
++   */
++  private Set<StripedDataStreamer> checkStreamers() throws IOException {
++    Set<StripedDataStreamer> newFailed = new HashSet<>();
++    for(StripedDataStreamer s : streamers) {
++      if (!s.isHealthy() && !failedStreamers.contains(s)) {
++        newFailed.add(s);
++      }
++    }
++
++    final int failCount = failedStreamers.size() + newFailed.size();
++    if (LOG.isDebugEnabled()) {
++      LOG.debug("checkStreamers: " + streamers);
++      LOG.debug("healthy streamer count=" + (numAllBlocks - failCount));
++      LOG.debug("original failed streamers: " + failedStreamers);
++      LOG.debug("newly failed streamers: " + newFailed);
++    }
++    if (failCount > (numAllBlocks - numDataBlocks)) {
++      throw new IOException("Failed: the number of failed blocks = "
++          + failCount + " > the number of data blocks = "
++          + (numAllBlocks - numDataBlocks));
++    }
++    return newFailed;
++  }
++
++  private void handleStreamerFailure(String err, Exception e)
++      throws IOException {
++    LOG.warn("Failed: " + err + ", " + this, e);
++    getCurrentStreamer().getErrorState().setInternalError();
++    getCurrentStreamer().close(true);
++    checkStreamers();
++    currentPacket = null;
++  }
++
++  private void replaceFailedStreamers() {
++    assert streamers.size() == numAllBlocks;
++    for (short i = 0; i < numAllBlocks; i++) {
++      final StripedDataStreamer oldStreamer = getStripedDataStreamer(i);
++      if (!oldStreamer.isHealthy()) {
++        StripedDataStreamer streamer = new StripedDataStreamer(oldStreamer.stat,
++            dfsClient, src, oldStreamer.progress,
++            oldStreamer.checksum4WriteBlock, cachingStrategy, byteArrayManager,
++            favoredNodes, i, coordinator);
++        streamers.set(i, streamer);
++        currentPackets[i] = null;
++        if (i == 0) {
++          this.streamer = streamer;
++        }
++        streamer.start();
++      }
++    }
++  }
++
++  private void waitEndBlocks(int i) throws IOException {
++    while (getStripedDataStreamer(i).isHealthy()) {
++      final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i);
++      if (b != null) {
++        StripedBlockUtil.checkBlocks(currentBlockGroup, i, b);
++        return;
++      }
++    }
++  }
++
++  private void allocateNewBlock() throws IOException {
++    if (currentBlockGroup != null) {
++      for (int i = 0; i < numAllBlocks; i++) {
++        // sync all the healthy streamers before writing to the new block
++        waitEndBlocks(i);
++      }
++    }
++    failedStreamers.clear();
++    // replace failed streamers
++    replaceFailedStreamers();
++
++    if (LOG.isDebugEnabled()) {
++      LOG.debug("Allocating new block group. The previous block group: "
++          + currentBlockGroup);
++    }
++
++    // TODO collect excludedNodes from all the data streamers
++    final LocatedBlock lb = addBlock(null, dfsClient, src, currentBlockGroup,
++        fileId, favoredNodes);
++    assert lb.isStriped();
++    if (lb.getLocations().length < numDataBlocks) {
++      throw new IOException("Failed to get " + numDataBlocks
++          + " nodes from namenode: blockGroupSize= " + numAllBlocks
++          + ", blocks.length= " + lb.getLocations().length);
++    }
++    // assign the new block to the current block group
++    currentBlockGroup = lb.getBlock();
++
++    final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
++        (LocatedStripedBlock) lb, cellSize, numDataBlocks,
++        numAllBlocks - numDataBlocks);
++    for (int i = 0; i < blocks.length; i++) {
++      StripedDataStreamer si = getStripedDataStreamer(i);
++      if (si.isHealthy()) { // skipping failed data streamer
++        if (blocks[i] == null) {
++          // Set exception and close streamer as there is no block locations
++          // found for the parity block.
++          LOG.warn("Failed to get block location for parity block, index=" + i);
++          si.getLastException().set(
++              new IOException("Failed to get following block, i=" + i));
++          si.getErrorState().setInternalError();
++          si.close(true);
++        } else {
++          coordinator.getFollowingBlocks().offer(i, blocks[i]);
++        }
++      }
++    }
++  }
++
++  private boolean shouldEndBlockGroup() {
++    return currentBlockGroup != null &&
++        currentBlockGroup.getNumBytes() == blockSize * numDataBlocks;
++  }
++
++  @Override
++  protected synchronized void writeChunk(byte[] bytes, int offset, int len,
++      byte[] checksum, int ckoff, int cklen) throws IOException {
++    final int index = getCurrentIndex();
++    final StripedDataStreamer current = getCurrentStreamer();
++    final int pos = cellBuffers.addTo(index, bytes, offset, len);
++    final boolean cellFull = pos == cellSize;
++
++    if (currentBlockGroup == null || shouldEndBlockGroup()) {
++      // the incoming data should belong to a new block. Allocate a new block.
++      allocateNewBlock();
++    }
++
++    currentBlockGroup.setNumBytes(currentBlockGroup.getNumBytes() + len);
++    if (current.isHealthy()) {
++      try {
++        super.writeChunk(bytes, offset, len, checksum, ckoff, cklen);
++      } catch(Exception e) {
++        handleStreamerFailure("offset=" + offset + ", length=" + len, e);
++      }
++    }
++
++    // Two extra steps are needed when a striping cell is full:
++    // 1. Forward the current index pointer
++    // 2. Generate parity packets if a full stripe of data cells are present
++    if (cellFull) {
++      int next = index + 1;
++      //When all data cells in a stripe are ready, we need to encode
++      //them and generate some parity cells. These cells will be
++      //converted to packets and put to their DataStreamer's queue.
++      if (next == numDataBlocks) {
++        cellBuffers.flipDataBuffers();
++        writeParityCells();
++        next = 0;
++        // check failure state for all the streamers. Bump GS if necessary
++        checkStreamerFailures();
++
++        // if this is the end of the block group, end each internal block
++        if (shouldEndBlockGroup()) {
++          for (int i = 0; i < numAllBlocks; i++) {
++            final StripedDataStreamer s = setCurrentStreamer(i);
++            if (s.isHealthy()) {
++              try {
++                endBlock();
++              } catch (IOException ignored) {}
++            }
++          }
++        }
++      }
++      setCurrentStreamer(next);
++    }
++  }
++
++  @Override
++  void enqueueCurrentPacketFull() throws IOException {
++    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
++            + " appendChunk={}, {}", currentPacket, src, getStreamer()
++            .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
++        getStreamer());
++    enqueueCurrentPacket();
++    adjustChunkBoundary();
++    // no need to end block here
++  }
++
++  private Set<StripedDataStreamer> markExternalErrorOnStreamers() {
++    Set<StripedDataStreamer> healthySet = new HashSet<>();
++    for (StripedDataStreamer streamer : streamers) {
++      if (streamer.isHealthy() &&
++          streamer.getStage() == BlockConstructionStage.DATA_STREAMING) {
++        streamer.setExternalError();
++        healthySet.add(streamer);
++      }
++    }
++    return healthySet;
++  }
++
++  /**
++   * Check and handle data streamer failures. This is called only when we have
++   * written a full stripe (i.e., enqueue all packets for a full stripe), or
++   * when we're closing the outputstream.
++   */
++  private void checkStreamerFailures() throws IOException {
++    Set<StripedDataStreamer> newFailed = checkStreamers();
++    if (newFailed.size() > 0) {
++      // for healthy streamers, wait till all of them have fetched the new block
++      // and flushed out all the enqueued packets.
++      flushAllInternals();
++    }
++    // get all the current failed streamers after the flush
++    newFailed = checkStreamers();
++    while (newFailed.size() > 0) {
++      failedStreamers.addAll(newFailed);
++      coordinator.clearFailureStates();
++
++      // mark all the healthy streamers as external error
++      Set<StripedDataStreamer> healthySet = markExternalErrorOnStreamers();
++
++      // we have newly failed streamers, update block for pipeline
++      final ExtendedBlock newBG = updateBlockForPipeline(healthySet);
++
++      // wait till all the healthy streamers to
++      // 1) get the updated block info
++      // 2) create new block outputstream
++      newFailed = waitCreatingNewStreams(healthySet);
++      if (newFailed.size() + failedStreamers.size() >
++          numAllBlocks - numDataBlocks) {
++        throw new IOException(
++            "Data streamers failed while creating new block streams: "
++                + newFailed + ". There are not enough healthy streamers.");
++      }
++      for (StripedDataStreamer failedStreamer : newFailed) {
++        assert !failedStreamer.isHealthy();
++      }
++
++      // TODO we can also succeed if all the failed streamers have not taken
++      // the updated block
++      if (newFailed.size() == 0) {
++        // reset external error state of all the streamers
++        for (StripedDataStreamer streamer : healthySet) {
++          assert streamer.isHealthy();
++          streamer.getErrorState().reset();
++        }
++        updatePipeline(newBG);
++      }
++      for (int i = 0; i < numAllBlocks; i++) {
++        coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0);
++      }
++    }
++  }
++
++  private int checkStreamerUpdates(Set<StripedDataStreamer> failed,
++      Set<StripedDataStreamer> streamers) {
++    for (StripedDataStreamer streamer : streamers) {
++      if (!coordinator.updateStreamerMap.containsKey(streamer)) {
++        if (!streamer.isHealthy() &&
++            coordinator.getNewBlocks().peek(streamer.getIndex()) != null) {
++          // this streamer had internal error before getting updated block
++          failed.add(streamer);
++        }
++      }
++    }
++    return coordinator.updateStreamerMap.size() + failed.size();
++  }
++
++  private Set<StripedDataStreamer> waitCreatingNewStreams(
++      Set<StripedDataStreamer> healthyStreamers) throws IOException {
++    Set<StripedDataStreamer> failed = new HashSet<>();
++    final int expectedNum = healthyStreamers.size();
++    final long socketTimeout = dfsClient.getConf().getSocketTimeout();
++    // the total wait time should be less than the socket timeout, otherwise
++    // a slow streamer may cause other streamers to timeout. here we wait for
++    // half of the socket timeout
++    long remaingTime = socketTimeout > 0 ? socketTimeout/2 : Long.MAX_VALUE;
++    final long waitInterval = 1000;
++    synchronized (coordinator) {
++      while (checkStreamerUpdates(failed, healthyStreamers) < expectedNum
++          && remaingTime > 0) {
++        try {
++          long start = Time.monotonicNow();
++          coordinator.wait(waitInterval);
++          remaingTime -= Time.monotonicNow() - start;
++        } catch (InterruptedException e) {
++          throw DFSUtilClient.toInterruptedIOException("Interrupted when waiting" +
++              " for results of updating striped streamers", e);
++        }
++      }
++    }
++    synchronized (coordinator) {
++      for (StripedDataStreamer streamer : healthyStreamers) {
++        if (!coordinator.updateStreamerMap.containsKey(streamer)) {
++          // close the streamer if it is too slow to create new connection
++          streamer.setStreamerAsClosed();
++          failed.add(streamer);
++        }
++      }
++    }
++    for (Map.Entry<StripedDataStreamer, Boolean> entry :
++        coordinator.updateStreamerMap.entrySet()) {
++      if (!entry.getValue()) {
++        failed.add(entry.getKey());
++      }
++    }
++    for (StripedDataStreamer failedStreamer : failed) {
++      healthyStreamers.remove(failedStreamer);
++    }
++    return failed;
++  }
++
++  /**
++   * Call {@link ClientProtocol#updateBlockForPipeline} and assign updated block
++   * to healthy streamers.
++   * @param healthyStreamers The healthy data streamers. These streamers join
++   *                         the failure handling.
++   */
++  private ExtendedBlock updateBlockForPipeline(
++      Set<StripedDataStreamer> healthyStreamers) throws IOException {
++    final LocatedBlock updated = dfsClient.namenode.updateBlockForPipeline(
++        currentBlockGroup, dfsClient.clientName);
++    final long newGS = updated.getBlock().getGenerationStamp();
++    ExtendedBlock newBlock = new ExtendedBlock(currentBlockGroup);
++    newBlock.setGenerationStamp(newGS);
++    final LocatedBlock[] updatedBlks = StripedBlockUtil.parseStripedBlockGroup(
++        (LocatedStripedBlock) updated, cellSize, numDataBlocks,
++        numAllBlocks - numDataBlocks);
++
++    for (int i = 0; i < numAllBlocks; i++) {
++      StripedDataStreamer si = getStripedDataStreamer(i);
++      if (healthyStreamers.contains(si)) {
++        final LocatedBlock lb = new LocatedBlock(new ExtendedBlock(newBlock),
++            null, null, null, -1, updated.isCorrupt(), null);
++        lb.setBlockToken(updatedBlks[i].getBlockToken());
++        coordinator.getNewBlocks().offer(i, lb);
++      }
++    }
++    return newBlock;
++  }
++
++  private void updatePipeline(ExtendedBlock newBG) throws IOException {
++    final DatanodeInfo[] newNodes = new DatanodeInfo[numAllBlocks];
++    final String[] newStorageIDs = new String[numAllBlocks];
++    for (int i = 0; i < numAllBlocks; i++) {
++      final StripedDataStreamer streamer = getStripedDataStreamer(i);
++      final DatanodeInfo[] nodes = streamer.getNodes();
++      final String[] storageIDs = streamer.getStorageIDs();
++      if (streamer.isHealthy() && nodes != null && storageIDs != null) {
++        newNodes[i] = nodes[0];
++        newStorageIDs[i] = storageIDs[0];
++      } else {
++        newNodes[i] = new DatanodeInfo(DatanodeID.EMPTY_DATANODE_ID);
++        newStorageIDs[i] = "";
++      }
++    }
++    dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup,
++        newBG, newNodes, newStorageIDs);
++    currentBlockGroup = newBG;
++  }
++
++  private int stripeDataSize() {
++    return numDataBlocks * cellSize;
++  }
++
++  @Override
++  public void hflush() {
++    throw new UnsupportedOperationException();
++  }
++
++  @Override
++  public void hsync() {
++    throw new UnsupportedOperationException();
++  }
++
++  @Override
++  protected synchronized void start() {
++    for (StripedDataStreamer streamer : streamers) {
++      streamer.start();
++    }
++  }
++
++  @Override
++  synchronized void abort() throws IOException {
++    if (isClosed()) {
++      return;
++    }
++    for (StripedDataStreamer streamer : streamers) {
++      streamer.getLastException().set(new IOException("Lease timeout of "
++          + (dfsClient.getConf().getHdfsTimeout()/1000) +
++          " seconds expired."));
++    }
++    closeThreads(true);
++    dfsClient.endFileLease(fileId);
++  }
++
++  @Override
++  boolean isClosed() {
++    if (closed) {
++      return true;
++    }
++    for(StripedDataStreamer s : streamers) {
++      if (!s.streamerClosed()) {
++        return false;
++      }
++    }
++    return true;
++  }
++
++  @Override
++  protected void closeThreads(boolean force) throws IOException {
++    final MultipleIOException.Builder b = new MultipleIOException.Builder();
++    try {
++      for (StripedDataStreamer streamer : streamers) {
++        try {
++          streamer.close(force);
++          streamer.join();
++          streamer.closeSocket();
++        } catch (Exception e) {
++          try {
++            handleStreamerFailure("force=" + force, e);
++          } catch (IOException ioe) {
++            b.add(ioe);
++          }
++        } finally {
++          streamer.setSocketToNull();
++        }
++      }
++    } finally {
++      setClosed();
++    }
++    final IOException ioe = b.build();
++    if (ioe != null) {
++      throw ioe;
++    }
++  }
++
++  private boolean generateParityCellsForLastStripe() {
++    final long currentBlockGroupBytes = currentBlockGroup == null ?
++        0 : currentBlockGroup.getNumBytes();
++    final long lastStripeSize = currentBlockGroupBytes % stripeDataSize();
++    if (lastStripeSize == 0) {
++      return false;
++    }
++
++    final long parityCellSize = lastStripeSize < cellSize?
++        lastStripeSize : cellSize;
++    final ByteBuffer[] buffers = cellBuffers.getBuffers();
++
++    for (int i = 0; i < numAllBlocks; i++) {
++      // Pad zero bytes to make all cells exactly the size of parityCellSize
++      // If internal block is smaller than parity block, pad zero bytes.
++      // Also pad zero bytes to all parity cells
++      final int position = buffers[i].position();
++      assert position <= parityCellSize : "If an internal block is smaller" +
++          " than parity block, then its last cell should be small than last" +
++          " parity cell";
++      for (int j = 0; j < parityCellSize - position; j++) {
++        buffers[i].put((byte) 0);
++      }
++      buffers[i].flip();
++    }
++    return true;
++  }
++
++  void writeParityCells() throws IOException {
++    final ByteBuffer[] buffers = cellBuffers.getBuffers();
++    //encode the data cells
++    encode(encoder, numDataBlocks, buffers);
++    for (int i = numDataBlocks; i < numAllBlocks; i++) {
++      writeParity(i, buffers[i], cellBuffers.getChecksumArray(i));
++    }
++    cellBuffers.clear();
++  }
++
++  void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf)
++      throws IOException {
++    final StripedDataStreamer current = setCurrentStreamer(index);
++    final int len = buffer.limit();
++
++    final long oldBytes = current.getBytesCurBlock();
++    if (current.isHealthy()) {
++      try {
++        DataChecksum sum = getDataChecksum();
++        sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0);
++        for (int i = 0; i < len; i += sum.getBytesPerChecksum()) {
++          int chunkLen = Math.min(sum.getBytesPerChecksum(), len - i);
++          int ckOffset = i / sum.getBytesPerChecksum() * getChecksumSize();
++          super.writeChunk(buffer.array(), i, chunkLen, checksumBuf, ckOffset,
++              getChecksumSize());
++        }
++      } catch(Exception e) {
++        handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
++      }
++    }
++  }
++
++  @Override
++  void setClosed() {
++    super.setClosed();
++    for (int i = 0; i < numAllBlocks; i++) {
++      getStripedDataStreamer(i).release();
++    }
++    cellBuffers.release();
++  }
++
++  @Override
++  protected synchronized void closeImpl() throws IOException {
++    if (isClosed()) {
++      final MultipleIOException.Builder b = new MultipleIOException.Builder();
++      for(int i = 0; i < streamers.size(); i++) {
++        final StripedDataStreamer si = getStripedDataStreamer(i);
++        try {
++          si.getLastException().check(true);
++        } catch (IOException e) {
++          b.add(e);
++        }
++      }
++      final IOException ioe = b.build();
++      if (ioe != null) {
++        throw ioe;
++      }
++      return;
++    }
++
++    try {
++      // flush from all upper layers
++      flushBuffer();
++      // if the last stripe is incomplete, generate and write parity cells
++      if (generateParityCellsForLastStripe()) {
++        writeParityCells();
++      }
++      enqueueAllCurrentPackets();
++
++      // flush all the data packets
++      flushAllInternals();
++      // check failures
++      checkStreamerFailures();
++
++      for (int i = 0; i < numAllBlocks; i++) {
++        final StripedDataStreamer s = setCurrentStreamer(i);
++        if (s.isHealthy()) {
++          try {
++            if (s.getBytesCurBlock() > 0) {
++              setCurrentPacketToEmpty();
++            }
++            // flush the last "close" packet to Datanode
++            flushInternal();
++          } catch(Exception e) {
++            // TODO for both close and endBlock, we currently do not handle
++            // failures when sending the last packet. We actually do not need to
++            // bump GS for this kind of failure. Thus counting the total number
++            // of failures may be good enough.
++          }
++        }
++      }
++
++      closeThreads(false);
++      TraceScope scope = dfsClient.getTracer().newScope("completeFile");
++      try {
++        completeFile(currentBlockGroup);
++      } finally {
++        scope.close();
++      }
++      dfsClient.endFileLease(fileId);
++    } catch (ClosedChannelException ignored) {
++    } finally {
++      setClosed();
++    }
++  }
++
++  private void enqueueAllCurrentPackets() throws IOException {
++    int idx = streamers.indexOf(getCurrentStreamer());
++    for(int i = 0; i < streamers.size(); i++) {
++      final StripedDataStreamer si = setCurrentStreamer(i);
++      if (si.isHealthy() && currentPacket != null) {
++        try {
++          enqueueCurrentPacket();
++        } catch (IOException e) {
++          handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e);
++        }
++      }
++    }
++    setCurrentStreamer(idx);
++  }
++
++  void flushAllInternals() throws IOException {
++    int current = getCurrentIndex();
++
++    for (int i = 0; i < numAllBlocks; i++) {
++      final StripedDataStreamer s = setCurrentStreamer(i);
++      if (s.isHealthy()) {
++        try {
++          // flush all data to Datanode
++          flushInternal();
++        } catch(Exception e) {
++          handleStreamerFailure("flushInternal " + s, e);
++        }
++      }
++    }
++    setCurrentStreamer(current);
++  }
++
++  static void sleep(long ms, String op) throws InterruptedIOException {
++    try {
++      Thread.sleep(ms);
++    } catch(InterruptedException ie) {
++      throw DFSUtilClient.toInterruptedIOException(
++          "Sleep interrupted during " + op, ie);
++    }
++  }
++
++  @Override
++  ExtendedBlock getBlock() {
++    return currentBlockGroup;
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 359886e,e275afb..f96ae65
--- 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
@@@ -53,6 -54,6 +54,7 @@@ import org.slf4j.LoggerFactory
  
  import javax.net.SocketFactory;
  import java.io.IOException;
++import java.io.InterruptedIOException;
  import java.io.UnsupportedEncodingException;
  import java.net.InetAddress;
  import java.net.InetSocketAddress;
@@@ -628,4 -652,4 +653,11 @@@ public class DFSUtilClient 
      return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
          + namenode.getHostName() + portString);
    }
++
++  public static InterruptedIOException toInterruptedIOException(String message,
++      InterruptedException e) {
++    final InterruptedIOException iioe = new InterruptedIOException(message);
++    iioe.initCause(e);
++    return iioe;
++  }
  }


[38/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 0000000,7a40d73..78eaa6c
mode 000000,100755..100755
--- 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
@@@ -1,0 -1,917 +1,982 @@@
+ /**
+  * 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.FileSystem;
+ 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.NotReplicatedYetException;
+ 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.core.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 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);
++        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 {
++      DataChecksum checksum, String[] favoredNodes, boolean createStreamer)
++      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);
++    if (createStreamer) {
++      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.newPathTraceScope("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);
++      final DFSOutputStream out;
++      if(stat.getErasureCodingPolicy() != null) {
++        out = new DFSStripedOutputStream(dfsClient, src, stat,
++            flag, progress, checksum, favoredNodes);
++      } else {
++        out = new DFSOutputStream(dfsClient, src, stat,
++            flag, progress, checksum, favoredNodes, true);
++      }
+       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 {
++      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.newPathTraceScope("newStreamForAppend", src);
++    if(stat.getErasureCodingPolicy() != null) {
++      throw new IOException("Not support appending to a striping layout file yet.");
++    }
+     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.newPathTraceScope("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=" + 
++      if (LOG.isDebugEnabled()) {
++        LOG.debug("WriteChunk allocating new packet seqno=" +
+             currentPacket.getSeqno() +
+             ", src=" + src +
+             ", packetSize=" + packetSize +
+             ", chunksPerPacket=" + chunksPerPacket +
 -            ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
++            ", bytesCurBlock=" + getStreamer().getBytesCurBlock() + ", " + this);
+       }
+     }
+ 
+     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(),
++            + " 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 {
++  void endBlock() throws IOException {
+     if (getStreamer().getBytesCurBlock() == blockSize) {
+       setCurrentPacketToEmpty();
+       enqueueCurrentPacket();
+       getStreamer().setBytesCurBlock(0);
+       lastFlushOffset = 0;
+     }
+   }
 -  
++
+   /**
+    * 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.newPathTraceScope("hflush", src);
+     try {
+       flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   @Override
+   public void hsync() throws IOException {
+     TraceScope scope =
+         dfsClient.newPathTraceScope("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.newPathTraceScope("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.newPathTraceScope("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 = dfsClient.getTracer().newScope("completeFile");
+       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();
++          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;
+   }
++
++  @Override
++  public String toString() {
++    return getClass().getSimpleName() + ":" + streamer;
++  }
++
++  static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient,
++      String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes)
++      throws IOException {
++    final DfsClientConf conf = dfsClient.getConf();
++    int retries = conf.getNumBlockWriteLocateFollowingRetry();
++    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
++    long localstart = Time.monotonicNow();
++    while (true) {
++      try {
++        return dfsClient.namenode.addBlock(src, dfsClient.clientName, prevBlock,
++            excludedNodes, fileId, 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;
++        }
++      }
++    }
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
index 0000000,9a8ca6f..191691b
mode 000000,100755..100755
--- 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
@@@ -1,0 -1,350 +1,364 @@@
+ /**
+  * 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.ByteBuffer;
+ 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.core.Span;
+ import org.apache.htrace.core.SpanId;
+ import org.apache.htrace.core.TraceScope;
+ 
+ /****************************************************************
+  * DFSPacket is used by DataStreamer and DFSOutputStream.
+  * DFSOutputStream generates packets and then ask DatStreamer
+  * to send them to datanodes.
+  ****************************************************************/
+ 
+ @InterfaceAudience.Private
 -class DFSPacket {
++public class DFSPacket {
+   public static final long HEART_BEAT_SEQNO = -1L;
+   private static SpanId[] EMPTY = new SpanId[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 SpanId[] traceParents = EMPTY;
+   private int traceParentsUsed;
+   private TraceScope scope;
+ 
+   /**
+    * 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,
++  public 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;
+   }
+ 
++  public synchronized void writeData(ByteBuffer inBuffer, int len)
++      throws ClosedChannelException {
++    checkBuffer();
++    len =  len > inBuffer.remaining() ? inBuffer.remaining() : len;
++    if (dataPos + len > buf.length) {
++      throw new BufferOverflowException();
++    }
++    for (int i = 0; i < len; i++) {
++      buf[dataPos + i] = inBuffer.get();
++    }
++    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)
++  public 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 {
++  public 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(){
++  boolean isLastPacketInBlock() {
+     return lastPacketInBlock;
+   }
+ 
+   /**
+    * get sequence number of this packet
+    *
+    * @return the sequence number of this packet
+    */
 -  long getSeqno(){
++  long getSeqno() {
+     return seqno;
+   }
+ 
+   /**
+    * get the number of chunks this packet contains
+    *
+    * @return the number of chunks in this packet
+    */
 -  synchronized int getNumChunks(){
++  synchronized int getNumChunks() {
+     return numChunks;
+   }
+ 
+   /**
+    * increase the number of chunks by one
+    */
 -  synchronized void incNumChunks(){
++  synchronized void incNumChunks() {
+     numChunks++;
+   }
+ 
+   /**
+    * get the maximum number of packets
+    *
+    * @return the maximum number of packets
+    */
 -  int getMaxChunks(){
++  int getMaxChunks() {
+     return maxChunks;
+   }
+ 
+   /**
+    * set if to sync block
+    *
+    * @param syncBlock if to sync block
+    */
 -  synchronized void setSyncBlock(boolean syncBlock){
++  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(SpanId id) {
+     if (!id.isValid()) {
+       return;
+     }
+     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 SpanId[] getTraceParents() {
+     // Remove duplicates from the array.
+     int len = traceParentsUsed;
+     Arrays.sort(traceParents, 0, len);
+     int i = 0, j = 0;
+     SpanId prevVal = SpanId.INVALID;
+     while (true) {
+       if (i == len) {
+         break;
+       }
+       SpanId val = traceParents[i];
+       if (!val.equals(prevVal)) {
+         traceParents[j] = val;
+         j++;
+         prevVal = val;
+       }
+       i++;
+     }
+     if (j < traceParents.length) {
+       traceParents = Arrays.copyOf(traceParents, j);
+       traceParentsUsed = traceParents.length;
+     }
+     return traceParents;
+   }
+ 
+   public void setTraceScope(TraceScope scope) {
+     this.scope = scope;
+   }
+ 
+   public TraceScope getTraceScope() {
+     return scope;
+   }
+ }


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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();
-  }
-}


[44/58] [abbrv] hadoop git commit: HADOOP-12447. Clean up some htrace integration issues (cmccabe)

Posted by zh...@apache.org.
HADOOP-12447. Clean up some htrace integration issues (cmccabe)


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

Branch: refs/heads/HDFS-7285
Commit: 850d679acb935a6a6b0e6cb6f69d998e99395468
Parents: a0b5a0a
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Sep 29 09:25:11 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Sep 29 09:25:11 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Tracing.md       | 15 +++++++--------
 .../hadoop/hdfs/client/HdfsClientConfigKeys.java     |  2 --
 .../java/org/apache/hadoop/hdfs/DFSConfigKeys.java   |  3 ---
 .../hdfs/server/namenode/NameNodeRpcServer.java      |  2 +-
 .../hadoop/hdfs/server/namenode/NamenodeFsck.java    |  5 ++++-
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml  |  8 --------
 .../blockmanagement/TestBlockTokenWithDFS.java       |  2 ++
 .../server/datanode/TestDataNodeVolumeFailure.java   |  2 ++
 .../org/apache/hadoop/tracing/TestTraceAdmin.java    |  8 ++++----
 9 files changed, 20 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md b/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
index 7897855..4cc6a07 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
@@ -85,8 +85,8 @@ You need to run the command against all servers if you want to update the config
 You need to specify the class name of span receiver as argument of `-class` option.
 You can specify the configuration associated with span receiver by `-Ckey=value` options.
 
-      $ hadoop trace -add -class LocalFileSpanReceiver -Cdfs.htrace.local-file-span-receiver.path=/tmp/htrace.out -host 192.168.56.2:9000
-      Added trace span receiver 2 with configuration dfs.htrace.local-file-span-receiver.path = /tmp/htrace.out
+      $ hadoop trace -add -class org.apache.htrace.core.LocalFileSpanReceiver -Chadoop.htrace.local.file.span.receiver.path=/tmp/htrace.out -host 192.168.56.2:9000
+      Added trace span receiver 2 with configuration hadoop.htrace.local.file.span.receiver.path = /tmp/htrace.out
 
       $ hadoop trace -list -host 192.168.56.2:9000
       ID  CLASS
@@ -137,8 +137,7 @@ which start tracing span before invoking HDFS shell command.
         FsShell shell = new FsShell();
         conf.setQuietMode(false);
         shell.setConf(conf);
-        Tracer tracer = new Tracer.Builder().
-            name("TracingFsShell).
+        Tracer tracer = new Tracer.Builder("TracingFsShell").
             conf(TraceUtils.wrapHadoopConf("tracing.fs.shell.htrace.", conf)).
             build();
         int res = 0;
@@ -177,15 +176,15 @@ ProbabilitySampler.
 
 ```xml
       <property>
-        <name>dfs.client.htrace.spanreceiver.classes</name>
+        <name>hadoop.htrace.span.receiver.classes</name>
         <value>LocalFileSpanReceiver</value>
       </property>
       <property>
-        <name>dfs.client.htrace.sampler</name>
+        <name>fs.client.htrace.sampler.classes</name>
         <value>ProbabilitySampler</value>
       </property>
       <property>
-        <name>dfs.client.htrace.sampler.fraction</name>
-        <value>0.5</value>
+        <name>fs.client.htrace.sampler.fraction</name>
+        <value>0.01</value>
       </property>
 ```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index fe31531..992c374 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -149,8 +149,6 @@ public interface HdfsClientConfigKeys {
       "dfs.client.test.drop.namenode.response.number";
   int     DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT = 0;
   String  DFS_CLIENT_LOCAL_INTERFACES = "dfs.client.local.interfaces";
-  // HDFS client HTrace configuration.
-  String  DFS_CLIENT_HTRACE_PREFIX = "dfs.client.htrace.";
   String  DFS_USER_HOME_DIR_PREFIX_KEY = "dfs.user.home.dir.prefix";
   String  DFS_USER_HOME_DIR_PREFIX_DEFAULT = "/user";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/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 b631955..055d7e3 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
@@ -64,9 +64,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
 
-  public static final String DFS_CLIENT_HTRACE_SAMPLER_CLASSES =
-      "dfs.client.htrace.sampler.classes";
-
   // HA related configuration
   public static final String  DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";
   public static final long    DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT = 50;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 79a3773..551a0e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -419,7 +419,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
         UnresolvedPathException.class);
     clientRpcServer.setTracer(nn.tracer);
     if (serviceRpcServer != null) {
-      clientRpcServer.setTracer(nn.tracer);
+      serviceRpcServer.setTracer(nn.tracer);
     }
  }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 6d40898..309012e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.tracing.TraceUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.htrace.core.Tracer;
 
@@ -202,7 +203,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     this.staleInterval =
         conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
-    this.tracer = new Tracer.Builder("NamenodeFsck").build();
+    this.tracer = new Tracer.Builder("NamenodeFsck").
+        conf(TraceUtils.wrapHadoopConf("namenode.fsck.htrace.", conf)).
+        build();
 
     for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
       String key = it.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index f1b855e..77b556e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2428,14 +2428,6 @@
 </property>
 
 <property>
-  <name>dfs.client.htrace.sampler.classes</name>
-  <value></value>
-  <description>
-    The class names of the HTrace Samplers to use for the HDFS client.
-  </description>
-</property>
-
-<property>
   <name>dfs.ha.zkfc.nn.http.timeout.ms</name>
   <value>20000</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index 50d548a..e39ab3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
@@ -161,6 +162,7 @@ public class TestBlockTokenWithDFS {
           setCachingStrategy(CachingStrategy.newDefaultStrategy()).
           setClientCacheContext(ClientContext.getFromConf(conf)).
           setConfiguration(conf).
+          setTracer(FsTracer.get(conf)).
           setRemotePeerFactory(new RemotePeerFactory() {
             @Override
             public Peer newConnectedPeer(InetSocketAddress addr,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index cb50edc..2c4fcc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
@@ -515,6 +516,7 @@ public class TestDataNodeVolumeFailure {
       setCachingStrategy(CachingStrategy.newDefaultStrategy()).
       setClientCacheContext(ClientContext.getFromConf(conf)).
       setConfiguration(conf).
+      setTracer(FsTracer.get(conf)).
       setRemotePeerFactory(new RemotePeerFactory() {
         @Override
         public Peer newConnectedPeer(InetSocketAddress addr,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/850d679a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
index b08866b..7e10d90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
@@ -73,10 +73,10 @@ public class TestTraceAdmin {
       Assert.assertEquals("ret:0, [no span receivers found]" + NEWLINE,
           runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster)));
       Assert.assertEquals("ret:0, Added trace span receiver 1 with " +
-          "configuration dfs.htrace.local-file-span-receiver.path = " + tracePath + NEWLINE,
+          "configuration hadoop.htrace.local.file.span.receiver.path = " + tracePath + NEWLINE,
           runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
               "-class", "org.apache.htrace.core.LocalFileSpanReceiver",
-              "-Cdfs.htrace.local-file-span-receiver.path=" + tracePath));
+              "-Chadoop.htrace.local.file.span.receiver.path=" + tracePath));
       String list =
           runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster));
       Assert.assertTrue(list.startsWith("ret:0"));
@@ -87,10 +87,10 @@ public class TestTraceAdmin {
       Assert.assertEquals("ret:0, [no span receivers found]" + NEWLINE,
           runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster)));
       Assert.assertEquals("ret:0, Added trace span receiver 2 with " +
-          "configuration dfs.htrace.local-file-span-receiver.path = " + tracePath + NEWLINE,
+          "configuration hadoop.htrace.local.file.span.receiver.path = " + tracePath + NEWLINE,
           runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
               "-class", "LocalFileSpanReceiver",
-              "-Cdfs.htrace.local-file-span-receiver.path=" + tracePath));
+              "-Chadoop.htrace.local.file.span.receiver.path=" + tracePath));
       Assert.assertEquals("ret:0, Removed trace span receiver 2" + NEWLINE,
           runTraceCommand(trace, "-remove", "2", "-host",
               getHostPortForNN(cluster)));


[27/58] [abbrv] hadoop git commit: YARN-4141. Runtime Application Priority change should not throw exception for applications at finishing states. Contributed by Sunil G

Posted by zh...@apache.org.
YARN-4141. Runtime Application Priority change should not throw exception for applications at finishing states. Contributed by Sunil G


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

Branch: refs/heads/HDFS-7285
Commit: 9f53a95ff624f66a774fe3defeea4a3454f4c4af
Parents: 3abbdc9
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Sep 28 22:55:20 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Sep 28 22:55:20 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/resourcemanager/ClientRMService.java | 30 +++++++++++-----
 .../resourcemanager/TestClientRMService.java    | 36 +++++++++++---------
 3 files changed, 44 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f53a95f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 54207aa..3745d55 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -904,6 +904,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4204. ConcurrentModificationException in FairSchedulerQueueInfo. (adhoot)
 
+    YARN-4141. Runtime Application Priority change should not throw exception
+    for applications at finishing states (Sunil G via jlowe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f53a95f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 02c6a5f..dad86f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -185,6 +185,12 @@ public class ClientRMService extends AbstractService implements
   private ReservationSystem reservationSystem;
   private ReservationInputValidator rValidator;
 
+  private static final EnumSet<RMAppState> COMPLETED_APP_STATES = EnumSet.of(
+      RMAppState.FINISHED, RMAppState.FINISHING, RMAppState.FAILED,
+      RMAppState.KILLED, RMAppState.FINAL_SAVING, RMAppState.KILLING);
+  private static final EnumSet<RMAppState> ACTIVE_APP_STATES = EnumSet.of(
+      RMAppState.ACCEPTED, RMAppState.RUNNING);
+
   public ClientRMService(RMContext rmContext, YarnScheduler scheduler,
       RMAppManager rmAppManager, ApplicationACLsManager applicationACLsManager,
       QueueACLsManager queueACLsManager,
@@ -1334,7 +1340,8 @@ public class ClientRMService extends AbstractService implements
           AuditConstants.UPDATE_APP_PRIORITY, "UNKNOWN", "ClientRMService",
           "Trying to update priority of an absent application", applicationId);
       throw new ApplicationNotFoundException(
-          "Trying to update priority o an absent application " + applicationId);
+          "Trying to update priority of an absent application "
+          + applicationId);
     }
 
     if (!checkAccess(callerUGI, application.getUser(),
@@ -1349,12 +1356,20 @@ public class ClientRMService extends AbstractService implements
           + ApplicationAccessType.MODIFY_APP.name() + " on " + applicationId));
     }
 
+    UpdateApplicationPriorityResponse response = recordFactory
+        .newRecordInstance(UpdateApplicationPriorityResponse.class);
     // Update priority only when app is tracked by the scheduler
-    if (!EnumSet.of(RMAppState.ACCEPTED, RMAppState.RUNNING).contains(
-        application.getState())) {
-      String msg =
-          "Application in " + application.getState()
-              + " state cannot be update priority.";
+    if (!ACTIVE_APP_STATES.contains(application.getState())) {
+      if (COMPLETED_APP_STATES.contains(application.getState())) {
+        // If Application is in any of the final states, change priority
+        // can be skipped rather throwing exception.
+        RMAuditLogger.logSuccess(callerUGI.getShortUserName(),
+            AuditConstants.UPDATE_APP_PRIORITY, "ClientRMService",
+            applicationId);
+        return response;
+      }
+      String msg = "Application in " + application.getState()
+          + " state cannot update priority.";
       RMAuditLogger
           .logFailure(callerUGI.getShortUserName(),
               AuditConstants.UPDATE_APP_PRIORITY, "UNKNOWN", "ClientRMService",
@@ -1374,9 +1389,6 @@ public class ClientRMService extends AbstractService implements
 
     RMAuditLogger.logSuccess(callerUGI.getShortUserName(),
         AuditConstants.UPDATE_APP_PRIORITY, "ClientRMService", applicationId);
-    UpdateApplicationPriorityResponse response =
-        recordFactory
-            .newRecordInstance(UpdateApplicationPriorityResponse.class);
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f53a95f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 39964da..49b5b55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -1335,7 +1335,7 @@ public class TestClientRMService {
   @Test(timeout = 120000)
   public void testUpdateApplicationPriorityRequest() throws Exception {
     int maxPriority = 10;
-    int appPriorty = 5;
+    int appPriority = 5;
     YarnConfiguration conf = new YarnConfiguration();
     conf.setInt(YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY,
         maxPriority);
@@ -1344,43 +1344,47 @@ public class TestClientRMService {
     rm.start();
 
     // Start app1 with appPriority 5
-    RMApp app1 = rm.submitApp(1024, Priority.newInstance(appPriorty));
+    RMApp app1 = rm.submitApp(1024, Priority.newInstance(appPriority));
 
     Assert.assertEquals("Incorrect priority has been set to application",
-        appPriorty, app1.getApplicationSubmissionContext().getPriority()
+        appPriority, app1.getApplicationSubmissionContext().getPriority()
             .getPriority());
 
-    appPriorty = 9;
+    appPriority = 9;
     ClientRMService rmService = rm.getClientRMService();
     UpdateApplicationPriorityRequest updateRequest =
         UpdateApplicationPriorityRequest.newInstance(app1.getApplicationId(),
-            Priority.newInstance(appPriorty));
+            Priority.newInstance(appPriority));
 
     rmService.updateApplicationPriority(updateRequest);
 
     Assert.assertEquals("Incorrect priority has been set to application",
-        appPriorty, app1.getApplicationSubmissionContext().getPriority()
+        appPriority, app1.getApplicationSubmissionContext().getPriority()
             .getPriority());
 
     rm.killApp(app1.getApplicationId());
     rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
 
+    appPriority = 8;
+    UpdateApplicationPriorityRequest updateRequestNew =
+        UpdateApplicationPriorityRequest.newInstance(app1.getApplicationId(),
+            Priority.newInstance(appPriority));
     // Update priority request for application in KILLED state
-    try {
-      rmService.updateApplicationPriority(updateRequest);
-      Assert.fail("Can not update priority for an application in KILLED state");
-    } catch (YarnException e) {
-      String msg =
-          "Application in " + app1.getState()
-              + " state cannot be update priority.";
-      Assert.assertTrue("", msg.contains(e.getMessage()));
-    }
+    rmService.updateApplicationPriority(updateRequestNew);
+
+    // Hence new priority should not be updated
+    Assert.assertNotEquals("Priority should not be updated as app is in KILLED state",
+        appPriority, app1.getApplicationSubmissionContext().getPriority()
+            .getPriority());
+    Assert.assertEquals("Priority should be same as old one before update",
+        9, app1.getApplicationSubmissionContext().getPriority()
+            .getPriority());
 
     // Update priority request for invalid application id.
     ApplicationId invalidAppId = ApplicationId.newInstance(123456789L, 3);
     updateRequest =
         UpdateApplicationPriorityRequest.newInstance(invalidAppId,
-            Priority.newInstance(appPriorty));
+            Priority.newInstance(appPriority));
     try {
       rmService.updateApplicationPriority(updateRequest);
       Assert


[16/58] [abbrv] hadoop git commit: HDFS-8740. Move DistributedFileSystem to hadoop-hdfs-client. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HDFS-8740. Move DistributedFileSystem 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/1c030c6e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1c030c6e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1c030c6e

Branch: refs/heads/HDFS-7285
Commit: 1c030c6e58dc83152f933323bb7743ad47f5af27
Parents: f0f984e
Author: Haohui Mai <wh...@apache.org>
Authored: Sun Sep 27 10:54:44 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Sun Sep 27 10:54:44 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DistributedFileSystem.java      | 2260 +++++++++++++++++
 .../client/impl/CorruptFileBlockIterator.java   |  105 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |    3 +
 .../hadoop/hdfs/DistributedFileSystem.java      | 2262 ------------------
 .../client/impl/CorruptFileBlockIterator.java   |  105 -
 5 files changed, 2368 insertions(+), 2367 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c030c6e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
new file mode 100644
index 0000000..88e6637
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -0,0 +1,2260 @@
+/**
+ * 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.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+
+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.BlockStoragePolicySpi;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSLinkResolver;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemLinkResolver;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
+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.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+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.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.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+
+/****************************************************************
+ * Implementation of the abstract FileSystem for the DFS system.
+ * This object is the way end-user code interacts with a Hadoop
+ * DistributedFileSystem.
+ *
+ *****************************************************************/
+@InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
+@InterfaceStability.Unstable
+public class DistributedFileSystem extends FileSystem {
+  private Path workingDir;
+  private URI uri;
+  private String homeDirPrefix =
+      HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT;
+
+  DFSClient dfs;
+  private boolean verifyChecksum = true;
+  
+  static{
+    HdfsConfigurationLoader.init();
+  }
+
+  public DistributedFileSystem() {
+  }
+
+  /**
+   * Return the protocol scheme for the FileSystem.
+   * <p/>
+   *
+   * @return <code>hdfs</code>
+   */
+  @Override
+  public String getScheme() {
+    return HdfsConstants.HDFS_URI_SCHEME;
+  }
+
+  @Override
+  public URI getUri() { return uri; }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    setConf(conf);
+
+    String host = uri.getHost();
+    if (host == null) {
+      throw new IOException("Incomplete HDFS URI, no host: "+ uri);
+    }
+    homeDirPrefix = conf.get(
+        HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_KEY,
+        HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT);
+
+    this.dfs = new DFSClient(uri, conf, statistics);
+    this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
+    this.workingDir = getHomeDirectory();
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Override
+  public long getDefaultBlockSize() {
+    return dfs.getConf().getDefaultBlockSize();
+  }
+
+  @Override
+  public short getDefaultReplication() {
+    return dfs.getConf().getDefaultReplication();
+  }
+
+  @Override
+  public void setWorkingDirectory(Path dir) {
+    String result = fixRelativePart(dir).toUri().getPath();
+    if (!DFSUtilClient.isValidName(result)) {
+      throw new IllegalArgumentException("Invalid DFS directory name " + 
+                                         result);
+    }
+    workingDir = fixRelativePart(dir);
+  }
+
+  @Override
+  public Path getHomeDirectory() {
+    return makeQualified(new Path(homeDirPrefix + "/"
+        + dfs.ugi.getShortUserName()));
+  }
+
+  /**
+   * Checks that the passed URI belongs to this filesystem and returns
+   * just the path component. Expects a URI with an absolute path.
+   * 
+   * @param file URI with absolute path
+   * @return path component of {file}
+   * @throws IllegalArgumentException if URI does not belong to this DFS
+   */
+  private String getPathName(Path file) {
+    checkPath(file);
+    String result = file.toUri().getPath();
+    if (!DFSUtilClient.isValidName(result)) {
+      throw new IllegalArgumentException("Pathname " + result + " from " +
+                                         file+" is not a valid DFS filename.");
+    }
+    return result;
+  }
+  
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+      long len) throws IOException {
+    if (file == null) {
+      return null;
+    }
+    return getFileBlockLocations(file.getPath(), start, len);
+  }
+  
+  @Override
+  public BlockLocation[] getFileBlockLocations(Path p, 
+      final long start, final long len) throws IOException {
+    statistics.incrementReadOps(1);
+    final Path absF = fixRelativePart(p);
+    return new FileSystemLinkResolver<BlockLocation[]>() {
+      @Override
+      public BlockLocation[] doCall(final Path p) throws IOException {
+        return dfs.getBlockLocations(getPathName(p), start, len);
+      }
+      @Override
+      public BlockLocation[] next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.getFileBlockLocations(p, start, len);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public void setVerifyChecksum(boolean verifyChecksum) {
+    this.verifyChecksum = verifyChecksum;
+  }
+
+  /** 
+   * Start the lease recovery of a file
+   *
+   * @param f a file
+   * @return true if the file is already closed
+   * @throws IOException if an error occurs
+   */
+  public boolean recoverLease(final Path f) throws IOException {
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<Boolean>() {
+      @Override
+      public Boolean doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.recoverLease(getPathName(p));
+      }
+      @Override
+      public Boolean next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          return myDfs.recoverLease(p);
+        }
+        throw new UnsupportedOperationException("Cannot recoverLease through" +
+            " a symlink to a non-DistributedFileSystem: " + f + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public FSDataInputStream open(Path f, final int bufferSize)
+      throws IOException {
+    statistics.incrementReadOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FSDataInputStream>() {
+      @Override
+      public FSDataInputStream doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        final DFSInputStream dfsis =
+          dfs.open(getPathName(p), bufferSize, verifyChecksum);
+        return dfs.createWrappedInputStream(dfsis);
+      }
+      @Override
+      public FSDataInputStream next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.open(p, bufferSize);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public FSDataOutputStream append(Path f, final int bufferSize,
+      final Progressable progress) throws IOException {
+    return append(f, EnumSet.of(CreateFlag.APPEND), bufferSize, progress);
+  }
+
+  /**
+   * Append to an existing file (optional operation).
+   * 
+   * @param f the existing file to be appended.
+   * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
+   *          to be present.
+   * @param bufferSize the size of the buffer to be used.
+   * @param progress for reporting progress if it is not null.
+   * @return Returns instance of {@link FSDataOutputStream}
+   * @throws IOException
+   */
+  public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
+      final int bufferSize, final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FSDataOutputStream>() {
+      @Override
+      public FSDataOutputStream doCall(final Path p)
+          throws IOException {
+        return dfs.append(getPathName(p), bufferSize, flag, progress,
+            statistics);
+      }
+      @Override
+      public FSDataOutputStream next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.append(p, bufferSize);
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * Append to an existing file (optional operation).
+   * 
+   * @param f the existing file to be appended.
+   * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
+   *          to be present.
+   * @param bufferSize the size of the buffer to be used.
+   * @param progress for reporting progress if it is not null.
+   * @param favoredNodes Favored nodes for new blocks
+   * @return Returns instance of {@link FSDataOutputStream}
+   * @throws IOException
+   */
+  public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
+      final int bufferSize, final Progressable progress,
+      final InetSocketAddress[] favoredNodes) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FSDataOutputStream>() {
+      @Override
+      public FSDataOutputStream doCall(final Path p)
+          throws IOException {
+        return dfs.append(getPathName(p), bufferSize, flag, progress,
+            statistics, favoredNodes);
+      }
+      @Override
+      public FSDataOutputStream next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.append(p, bufferSize);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    return this.create(f, permission,
+        overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+            : EnumSet.of(CreateFlag.CREATE), bufferSize, replication,
+        blockSize, progress, null);
+  }
+
+  /**
+   * Same as  
+   * {@link #create(Path, FsPermission, boolean, int, short, long, 
+   * Progressable)} 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. And with favored nodes, blocks will be pinned
+   * on the datanodes to prevent balancing move the block. HDFS could move the
+   * blocks during replication, to move the blocks from favored nodes. A value
+   * of null means no favored nodes for this create
+   */
+  public HdfsDataOutputStream create(final Path f,
+      final FsPermission permission, final boolean overwrite,
+      final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress, final InetSocketAddress[] favoredNodes)
+          throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<HdfsDataOutputStream>() {
+      @Override
+      public HdfsDataOutputStream doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        final DFSOutputStream out = dfs.create(getPathName(f), permission,
+            overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+                : EnumSet.of(CreateFlag.CREATE),
+            true, replication, blockSize, progress, bufferSize, null,
+            favoredNodes);
+        return dfs.createWrappedOutputStream(out, statistics);
+      }
+      @Override
+      public HdfsDataOutputStream next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          return myDfs.create(p, permission, overwrite, bufferSize, replication,
+              blockSize, progress, favoredNodes);
+        }
+        throw new UnsupportedOperationException("Cannot create with" +
+            " favoredNodes through a symlink to a non-DistributedFileSystem: "
+            + f + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public FSDataOutputStream create(final Path f, final FsPermission permission,
+    final EnumSet<CreateFlag> cflags, final int bufferSize,
+    final short replication, final long blockSize, final Progressable progress,
+    final ChecksumOpt checksumOpt) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FSDataOutputStream>() {
+      @Override
+      public FSDataOutputStream doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
+                cflags, replication, blockSize, progress, bufferSize,
+                checksumOpt);
+        return dfs.createWrappedOutputStream(dfsos, statistics);
+      }
+      @Override
+      public FSDataOutputStream next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.create(p, permission, cflags, bufferSize,
+            replication, blockSize, progress, checksumOpt);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  protected HdfsDataOutputStream primitiveCreate(Path f,
+    FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
+    short replication, long blockSize, Progressable progress,
+    ChecksumOpt checksumOpt) throws IOException {
+    statistics.incrementWriteOps(1);
+    final DFSOutputStream dfsos = dfs.primitiveCreate(
+      getPathName(fixRelativePart(f)),
+      absolutePermission, flag, true, replication, blockSize,
+      progress, bufferSize, checksumOpt);
+    return dfs.createWrappedOutputStream(dfsos, statistics);
+  }
+
+  /**
+   * Same as create(), except fails if parent directory doesn't already exist.
+   */
+  @Override
+  @SuppressWarnings("deprecation")
+  public FSDataOutputStream createNonRecursive(final Path f,
+      final FsPermission permission, final EnumSet<CreateFlag> flag,
+      final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+    if (flag.contains(CreateFlag.OVERWRITE)) {
+      flag.add(CreateFlag.CREATE);
+    }
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FSDataOutputStream>() {
+      @Override
+      public FSDataOutputStream doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
+          flag, false, replication, blockSize, progress, bufferSize, null);
+        return dfs.createWrappedOutputStream(dfsos, statistics);
+      }
+
+      @Override
+      public FSDataOutputStream next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.createNonRecursive(p, permission, flag, bufferSize,
+            replication, blockSize, progress);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public boolean setReplication(Path src, 
+                                final short replication
+                               ) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(src);
+    return new FileSystemLinkResolver<Boolean>() {
+      @Override
+      public Boolean doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.setReplication(getPathName(p), replication);
+      }
+      @Override
+      public Boolean next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.setReplication(p, replication);
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * Set the source path to the specified storage policy.
+   *
+   * @param src The source path referring to either a directory or a file.
+   * @param policyName The name of the storage policy.
+   */
+  @Override
+  public void setStoragePolicy(final Path src, final String policyName)
+      throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(src);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.setStoragePolicy(getPathName(p), policyName);
+        return null;
+      }
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        fs.setStoragePolicy(p, policyName);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException {
+    statistics.incrementReadOps(1);
+    Path absF = fixRelativePart(path);
+
+    return new FileSystemLinkResolver<BlockStoragePolicySpi>() {
+      @Override
+      public BlockStoragePolicySpi doCall(final Path p) throws IOException {
+        return getClient().getStoragePolicy(getPathName(p));
+      }
+
+      @Override
+      public BlockStoragePolicySpi next(final FileSystem fs, final Path p)
+          throws IOException, UnresolvedLinkException {
+        return fs.getStoragePolicy(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public Collection<BlockStoragePolicy> getAllStoragePolicies()
+      throws IOException {
+    return Arrays.asList(dfs.getStoragePolicies());
+  }
+
+  /**
+   * Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
+   * @return
+   * @throws IOException
+   */
+  @Deprecated
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    statistics.incrementReadOps(1);
+    return dfs.getStoragePolicies();
+  }
+
+  /**
+   * Move blocks from srcs to trg and delete srcs afterwards.
+   * The file block sizes must be the same.
+   * 
+   * @param trg existing file to append to
+   * @param psrcs list of files (same block size, same replication)
+   * @throws IOException
+   */
+  @Override
+  public void concat(Path trg, Path [] psrcs) throws IOException {
+    statistics.incrementWriteOps(1);
+    // Make target absolute
+    Path absF = fixRelativePart(trg);
+    // Make all srcs absolute
+    Path[] srcs = new Path[psrcs.length];
+    for (int i=0; i<psrcs.length; i++) {
+      srcs[i] = fixRelativePart(psrcs[i]);
+    }
+    // Try the concat without resolving any links
+    String[] srcsStr = new String[psrcs.length];
+    try {
+      for (int i=0; i<psrcs.length; i++) {
+        srcsStr[i] = getPathName(srcs[i]);
+      }
+      dfs.concat(getPathName(trg), srcsStr);
+    } catch (UnresolvedLinkException e) {
+      // Exception could be from trg or any src.
+      // Fully resolve trg and srcs. Fail if any of them are a symlink.
+      FileStatus stat = getFileLinkStatus(absF);
+      if (stat.isSymlink()) {
+        throw new IOException("Cannot concat with a symlink target: "
+            + trg + " -> " + stat.getPath());
+      }
+      absF = fixRelativePart(stat.getPath());
+      for (int i=0; i<psrcs.length; i++) {
+        stat = getFileLinkStatus(srcs[i]);
+        if (stat.isSymlink()) {
+          throw new IOException("Cannot concat with a symlink src: "
+              + psrcs[i] + " -> " + stat.getPath());
+        }
+        srcs[i] = fixRelativePart(stat.getPath());
+      }
+      // Try concat again. Can still race with another symlink.
+      for (int i=0; i<psrcs.length; i++) {
+        srcsStr[i] = getPathName(srcs[i]);
+      }
+      dfs.concat(getPathName(absF), srcsStr);
+    }
+  }
+
+  
+  @SuppressWarnings("deprecation")
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    statistics.incrementWriteOps(1);
+
+    final Path absSrc = fixRelativePart(src);
+    final Path absDst = fixRelativePart(dst);
+
+    // Try the rename without resolving first
+    try {
+      return dfs.rename(getPathName(absSrc), getPathName(absDst));
+    } catch (UnresolvedLinkException e) {
+      // Fully resolve the source
+      final Path source = getFileLinkStatus(absSrc).getPath();
+      // Keep trying to resolve the destination
+      return new FileSystemLinkResolver<Boolean>() {
+        @Override
+        public Boolean doCall(final Path p)
+            throws IOException, UnresolvedLinkException {
+          return dfs.rename(getPathName(source), getPathName(p));
+        }
+        @Override
+        public Boolean next(final FileSystem fs, final Path p)
+            throws IOException {
+          // Should just throw an error in FileSystem#checkPath
+          return doCall(p);
+        }
+      }.resolve(this, absDst);
+    }
+  }
+
+  /** 
+   * This rename operation is guaranteed to be atomic.
+   */
+  @SuppressWarnings("deprecation")
+  @Override
+  public void rename(Path src, Path dst, final Options.Rename... options)
+      throws IOException {
+    statistics.incrementWriteOps(1);
+    final Path absSrc = fixRelativePart(src);
+    final Path absDst = fixRelativePart(dst);
+    // Try the rename without resolving first
+    try {
+      dfs.rename(getPathName(absSrc), getPathName(absDst), options);
+    } catch (UnresolvedLinkException e) {
+      // Fully resolve the source
+      final Path source = getFileLinkStatus(absSrc).getPath();
+      // Keep trying to resolve the destination
+      new FileSystemLinkResolver<Void>() {
+        @Override
+        public Void doCall(final Path p)
+            throws IOException, UnresolvedLinkException {
+          dfs.rename(getPathName(source), getPathName(p), options);
+          return null;
+        }
+        @Override
+        public Void next(final FileSystem fs, final Path p)
+            throws IOException {
+          // Should just throw an error in FileSystem#checkPath
+          return doCall(p);
+        }
+      }.resolve(this, absDst);
+    }
+  }
+
+  @Override
+  public boolean truncate(Path f, final long newLength) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<Boolean>() {
+      @Override
+      public Boolean doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.truncate(getPathName(p), newLength);
+      }
+      @Override
+      public Boolean next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.truncate(p, newLength);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public boolean delete(Path f, final boolean recursive) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<Boolean>() {
+      @Override
+      public Boolean doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.delete(getPathName(p), recursive);
+      }
+      @Override
+      public Boolean next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.delete(p, recursive);
+      }
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<ContentSummary>() {
+      @Override
+      public ContentSummary doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.getContentSummary(getPathName(p));
+      }
+      @Override
+      public ContentSummary next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.getContentSummary(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  /** Set a directory's quotas
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long, StorageType)
+   */
+  public void setQuota(Path src, final long namespaceQuota,
+      final long storagespaceQuota) throws IOException {
+    Path absF = fixRelativePart(src);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota);
+        return null;
+      }
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        // setQuota is not defined in FileSystem, so we only can resolve
+        // within this DFS
+        return doCall(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * Set the per type storage quota of a directory.
+   *
+   * @param src target directory whose quota is to be modified.
+   * @param type storage type of the specific storage type quota to be modified.
+   * @param quota value of the specific storage type quota to be modified.
+   * Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type.
+   */
+  public void setQuotaByStorageType(
+    Path src, final StorageType type, final long quota)
+    throws IOException {
+    Path absF = fixRelativePart(src);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+        throws IOException, UnresolvedLinkException {
+        dfs.setQuotaByStorageType(getPathName(p), type, quota);
+        return null;
+      }
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+        throws IOException {
+        // setQuotaByStorageType is not defined in FileSystem, so we only can resolve
+        // within this DFS
+        return doCall(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  private FileStatus[] listStatusInternal(Path p) throws IOException {
+    String src = getPathName(p);
+
+    // fetch the first batch of entries in the directory
+    DirectoryListing thisListing = dfs.listPaths(
+        src, HdfsFileStatus.EMPTY_NAME);
+
+    if (thisListing == null) { // the directory does not exist
+      throw new FileNotFoundException("File " + p + " does not exist.");
+    }
+    
+    HdfsFileStatus[] partialListing = thisListing.getPartialListing();
+    if (!thisListing.hasMore()) { // got all entries of the directory
+      FileStatus[] stats = new FileStatus[partialListing.length];
+      for (int i = 0; i < partialListing.length; i++) {
+        stats[i] = partialListing[i].makeQualified(getUri(), p);
+      }
+      statistics.incrementReadOps(1);
+      return stats;
+    }
+
+    // The directory size is too big that it needs to fetch more
+    // estimate the total number of entries in the directory
+    int totalNumEntries =
+      partialListing.length + thisListing.getRemainingEntries();
+    ArrayList<FileStatus> listing =
+      new ArrayList<FileStatus>(totalNumEntries);
+    // add the first batch of entries to the array list
+    for (HdfsFileStatus fileStatus : partialListing) {
+      listing.add(fileStatus.makeQualified(getUri(), p));
+    }
+    statistics.incrementLargeReadOps(1);
+ 
+    // now fetch more entries
+    do {
+      thisListing = dfs.listPaths(src, thisListing.getLastName());
+ 
+      if (thisListing == null) { // the directory is deleted
+        throw new FileNotFoundException("File " + p + " does not exist.");
+      }
+ 
+      partialListing = thisListing.getPartialListing();
+      for (HdfsFileStatus fileStatus : partialListing) {
+        listing.add(fileStatus.makeQualified(getUri(), p));
+      }
+      statistics.incrementLargeReadOps(1);
+    } while (thisListing.hasMore());
+ 
+    return listing.toArray(new FileStatus[listing.size()]);
+  }
+
+  /**
+   * List all the entries of a directory
+   *
+   * Note that this operation is not atomic for a large directory.
+   * The entries of a directory may be fetched from NameNode multiple times.
+   * It only guarantees that  each name occurs once if a directory
+   * undergoes changes between the calls.
+   */
+  @Override
+  public FileStatus[] listStatus(Path p) throws IOException {
+    Path absF = fixRelativePart(p);
+    return new FileSystemLinkResolver<FileStatus[]>() {
+      @Override
+      public FileStatus[] doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return listStatusInternal(p);
+      }
+      @Override
+      public FileStatus[] next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.listStatus(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
+      final PathFilter filter)
+  throws IOException {
+    Path absF = fixRelativePart(p);
+    return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() {
+      @Override
+      public RemoteIterator<LocatedFileStatus> doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return new DirListingIterator<LocatedFileStatus>(p, filter, true);
+      }
+
+      @Override
+      public RemoteIterator<LocatedFileStatus> next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          return ((DistributedFileSystem)fs).listLocatedStatus(p, filter);
+        }
+        // symlink resolution for this methos does not work cross file systems
+        // because it is a protected method.
+        throw new IOException("Link resolution does not work with multiple " +
+            "file systems for listLocatedStatus(): " + p);
+      }
+    }.resolve(this, absF);
+  }
+
+
+  /**
+   * Returns a remote iterator so that followup calls are made on demand
+   * while consuming the entries. This reduces memory consumption during
+   * listing of a large directory.
+   *
+   * @param p target path
+   * @return remote iterator
+   */
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(final Path p)
+  throws IOException {
+    Path absF = fixRelativePart(p);
+    return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() {
+      @Override
+      public RemoteIterator<FileStatus> doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return new DirListingIterator<FileStatus>(p, false);
+      }
+
+      @Override
+      public RemoteIterator<FileStatus> next(final FileSystem fs, final Path p)
+          throws IOException {
+          return ((DistributedFileSystem)fs).listStatusIterator(p);
+      }
+    }.resolve(this, absF);
+
+  }
+
+  /**
+   * This class defines an iterator that returns
+   * the file status of each file/subdirectory of a directory
+   * 
+   * if needLocation, status contains block location if it is a file
+   * throws a RuntimeException with the error as its cause.
+   * 
+   * @param <T> the type of the file status
+   */
+  private class  DirListingIterator<T extends FileStatus>
+  implements RemoteIterator<T> {
+    private DirectoryListing thisListing;
+    private int i;
+    private Path p;
+    private String src;
+    private T curStat = null;
+    private PathFilter filter;
+    private boolean needLocation;
+
+    private DirListingIterator(Path p, PathFilter filter,
+        boolean needLocation) throws IOException {
+      this.p = p;
+      this.src = getPathName(p);
+      this.filter = filter;
+      this.needLocation = needLocation;
+      // fetch the first batch of entries in the directory
+      thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME,
+          needLocation);
+      statistics.incrementReadOps(1);
+      if (thisListing == null) { // the directory does not exist
+        throw new FileNotFoundException("File " + p + " does not exist.");
+      }
+      i = 0;
+    }
+
+    private DirListingIterator(Path p, boolean needLocation)
+        throws IOException {
+      this(p, null, needLocation);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public boolean hasNext() throws IOException {
+      while (curStat == null && hasNextNoFilter()) {
+        T next;
+        HdfsFileStatus fileStat = thisListing.getPartialListing()[i++];
+        if (needLocation) {
+          next = (T)((HdfsLocatedFileStatus)fileStat)
+              .makeQualifiedLocated(getUri(), p);
+        } else {
+          next = (T)fileStat.makeQualified(getUri(), p);
+        }
+          // apply filter if not null
+        if (filter == null || filter.accept(next.getPath())) {
+          curStat = next;
+        }
+      }
+      return curStat != null;
+    }
+      
+    /** Check if there is a next item before applying the given filter */
+    private boolean hasNextNoFilter() throws IOException {
+      if (thisListing == null) {
+        return false;
+      }
+      if (i >= thisListing.getPartialListing().length
+          && thisListing.hasMore()) { 
+        // current listing is exhausted & fetch a new listing
+        thisListing = dfs.listPaths(src, thisListing.getLastName(),
+            needLocation);
+        statistics.incrementReadOps(1);
+        if (thisListing == null) {
+          return false;
+        }
+        i = 0;
+      }
+      return (i < thisListing.getPartialListing().length);
+    }
+
+    @Override
+    public T next() throws IOException {
+      if (hasNext()) {
+        T tmp = curStat;
+        curStat = null;
+        return tmp;
+      } 
+      throw new java.util.NoSuchElementException("No more entry in " + p);
+    }
+  }
+  
+  /**
+   * Create a directory, only when the parent directories exist.
+   *
+   * See {@link FsPermission#applyUMask(FsPermission)} for details of how
+   * the permission is applied.
+   *
+   * @param f           The path to create
+   * @param permission  The permission.  See FsPermission#applyUMask for 
+   *                    details about how this is used to calculate the
+   *                    effective permission.
+   */
+  public boolean mkdir(Path f, FsPermission permission) throws IOException {
+    return mkdirsInternal(f, permission, false);
+  }
+
+  /**
+   * Create a directory and its parent directories.
+   *
+   * See {@link FsPermission#applyUMask(FsPermission)} for details of how
+   * the permission is applied.
+   *
+   * @param f           The path to create
+   * @param permission  The permission.  See FsPermission#applyUMask for 
+   *                    details about how this is used to calculate the
+   *                    effective permission.
+   */
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    return mkdirsInternal(f, permission, true);
+  }
+
+  private boolean mkdirsInternal(Path f, final FsPermission permission,
+      final boolean createParent) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<Boolean>() {
+      @Override
+      public Boolean doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.mkdirs(getPathName(p), permission, createParent);
+      }
+
+      @Override
+      public Boolean next(final FileSystem fs, final Path p)
+          throws IOException {
+        // FileSystem doesn't have a non-recursive mkdir() method
+        // Best we can do is error out
+        if (!createParent) {
+          throw new IOException("FileSystem does not support non-recursive"
+              + "mkdir");
+        }
+        return fs.mkdirs(p, permission);
+      }
+    }.resolve(this, absF);
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
+    throws IOException {
+    statistics.incrementWriteOps(1);
+    return dfs.primitiveMkdir(getPathName(f), absolutePermission);
+  }
+
+ 
+  @Override
+  public void close() throws IOException {
+    try {
+      dfs.closeOutputStreams(false);
+      super.close();
+    } finally {
+      dfs.close();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "DFS[" + dfs + "]";
+  }
+
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  public DFSClient getClient() {
+    return dfs;
+  }        
+  
+  @Override
+  public FsStatus getStatus(Path p) throws IOException {
+    statistics.incrementReadOps(1);
+    return dfs.getDiskStatus();
+  }
+
+  /**
+   * Returns count of blocks with no good replicas left. Normally should be
+   * zero.
+   * 
+   * @throws IOException
+   */
+  public long getMissingBlocksCount() throws IOException {
+    return dfs.getMissingBlocksCount();
+  }
+
+  /**
+   * Returns count of blocks with replication factor 1 and have
+   * lost the only replica.
+   *
+   * @throws IOException
+   */
+  public long getMissingReplOneBlocksCount() throws IOException {
+    return dfs.getMissingReplOneBlocksCount();
+  }
+
+  /**
+   * Returns count of blocks with one of more replica missing.
+   * 
+   * @throws IOException
+   */
+  public long getUnderReplicatedBlocksCount() throws IOException {
+    return dfs.getUnderReplicatedBlocksCount();
+  }
+
+  /**
+   * Returns count of blocks with at least one replica marked corrupt.
+   * 
+   * @throws IOException
+   */
+  public long getCorruptBlocksCount() throws IOException {
+    return dfs.getCorruptBlocksCount();
+  }
+
+  @Override
+  public RemoteIterator<Path> listCorruptFileBlocks(Path path)
+    throws IOException {
+    return new CorruptFileBlockIterator(dfs, path);
+  }
+
+  /** @return datanode statistics. */
+  public DatanodeInfo[] getDataNodeStats() throws IOException {
+    return getDataNodeStats(DatanodeReportType.ALL);
+  }
+
+  /** @return datanode statistics for the given type. */
+  public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type
+      ) throws IOException {
+    return dfs.datanodeReport(type);
+  }
+
+  /**
+   * Enter, leave or get safe mode.
+   *  
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
+   *    HdfsConstants.SafeModeAction,boolean)
+   */
+  public boolean setSafeMode(HdfsConstants.SafeModeAction action) 
+  throws IOException {
+    return setSafeMode(action, false);
+  }
+
+  /**
+   * Enter, leave or get safe mode.
+   * 
+   * @param action
+   *          One of SafeModeAction.ENTER, SafeModeAction.LEAVE and
+   *          SafeModeAction.GET
+   * @param isChecked
+   *          If true check only for Active NNs status, else check first NN's
+   *          status
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(SafeModeAction, boolean)
+   */
+  public boolean setSafeMode(HdfsConstants.SafeModeAction action,
+      boolean isChecked) throws IOException {
+    return dfs.setSafeMode(action, isChecked);
+  }
+
+  /**
+   * Save namespace image.
+   *
+   * @param timeWindow NameNode can ignore this command if the latest
+   *                   checkpoint was done within the given time period (in
+   *                   seconds).
+   * @return true if a new checkpoint has been made
+   * @see ClientProtocol#saveNamespace(long, long)
+   */
+  public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+    return dfs.saveNamespace(timeWindow, txGap);
+  }
+
+  /**
+   * Save namespace image. NameNode always does the checkpoint.
+   */
+  public void saveNamespace() throws IOException {
+    saveNamespace(0, 0);
+  }
+
+  /**
+   * Rolls the edit log on the active NameNode.
+   * Requires super-user privileges.
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits()
+   * @return the transaction ID of the newly created segment
+   */
+  public long rollEdits() throws AccessControlException, IOException {
+    return dfs.rollEdits();
+  }
+
+  /**
+   * enable/disable/check restoreFaileStorage
+   * 
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
+   */
+  public boolean restoreFailedStorage(String arg)
+      throws AccessControlException, IOException {
+    return dfs.restoreFailedStorage(arg);
+  }
+  
+
+  /**
+   * Refreshes the list of hosts and excluded hosts from the configured 
+   * files.  
+   */
+  public void refreshNodes() throws IOException {
+    dfs.refreshNodes();
+  }
+
+  /**
+   * Finalize previously upgraded files system state.
+   * @throws IOException
+   */
+  public void finalizeUpgrade() throws IOException {
+    dfs.finalizeUpgrade();
+  }
+
+  /**
+   * Rolling upgrade: prepare/finalize/query.
+   */
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+      throws IOException {
+    return dfs.rollingUpgrade(action);
+  }
+
+  /*
+   * Requests the namenode to dump data strcutures into specified 
+   * file.
+   */
+  public void metaSave(String pathname) throws IOException {
+    dfs.metaSave(pathname);
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    return dfs.getServerDefaults();
+  }
+
+  /**
+   * Returns the stat information about the file.
+   * @throws FileNotFoundException if the file does not exist.
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FileStatus>() {
+      @Override
+      public FileStatus doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
+        if (fi != null) {
+          return fi.makeQualified(getUri(), p);
+        } else {
+          throw new FileNotFoundException("File does not exist: " + p);
+        }
+      }
+      @Override
+      public FileStatus next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.getFileStatus(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public void createSymlink(final Path target, final Path link,
+      final boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, UnsupportedFileSystemException, 
+      IOException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
+    statistics.incrementWriteOps(1);
+    final Path absF = fixRelativePart(link);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        dfs.createSymlink(target.toString(), getPathName(p), createParent);
+        return null;
+      }
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException, UnresolvedLinkException {
+        fs.createSymlink(target, p, createParent);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public boolean supportsSymlinks() {
+    return true;
+  }
+
+  @Override
+  public FileStatus getFileLinkStatus(final Path f)
+      throws AccessControlException, FileNotFoundException,
+      UnsupportedFileSystemException, IOException {
+    statistics.incrementReadOps(1);
+    final Path absF = fixRelativePart(f);
+    FileStatus status = new FileSystemLinkResolver<FileStatus>() {
+      @Override
+      public FileStatus doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
+        if (fi != null) {
+          return fi.makeQualified(getUri(), p);
+        } else {
+          throw new FileNotFoundException("File does not exist: " + p);
+        }
+      }
+      @Override
+      public FileStatus next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        return fs.getFileLinkStatus(p);
+      }
+    }.resolve(this, absF);
+    // Fully-qualify the symlink
+    if (status.isSymlink()) {
+      Path targetQual = FSLinkResolver.qualifySymlinkTarget(this.getUri(),
+          status.getPath(), status.getSymlink());
+      status.setSymlink(targetQual);
+    }
+    return status;
+  }
+
+  @Override
+  public Path getLinkTarget(final Path f) throws AccessControlException,
+      FileNotFoundException, UnsupportedFileSystemException, IOException {
+    statistics.incrementReadOps(1);
+    final Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<Path>() {
+      @Override
+      public Path doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
+        if (fi != null) {
+          return fi.makeQualified(getUri(), p).getSymlink();
+        } else {
+          throw new FileNotFoundException("File does not exist: " + p);
+        }
+      }
+      @Override
+      public Path next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        return fs.getLinkTarget(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  protected Path resolveLink(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    String target = dfs.getLinkTarget(getPathName(fixRelativePart(f)));
+    if (target == null) {
+      throw new FileNotFoundException("File does not exist: " + f.toString());
+    }
+    return new Path(target);
+  }
+
+  @Override
+  public FileChecksum getFileChecksum(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FileChecksum>() {
+      @Override
+      public FileChecksum doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
+      }
+
+      @Override
+      public FileChecksum next(final FileSystem fs, final Path p)
+          throws IOException {
+        return fs.getFileChecksum(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public FileChecksum getFileChecksum(Path f, final long length)
+      throws IOException {
+    statistics.incrementReadOps(1);
+    Path absF = fixRelativePart(f);
+    return new FileSystemLinkResolver<FileChecksum>() {
+      @Override
+      public FileChecksum doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.getFileChecksum(getPathName(p), length);
+      }
+
+      @Override
+      public FileChecksum next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          return ((DistributedFileSystem) fs).getFileChecksum(p, length);
+        } else {
+          throw new UnsupportedFileSystemException(
+              "getFileChecksum(Path, long) is not supported by "
+                  + fs.getClass().getSimpleName()); 
+        }
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public void setPermission(Path p, final FsPermission permission
+      ) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(p);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.setPermission(getPathName(p), permission);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        fs.setPermission(p, permission);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public void setOwner(Path p, final String username, final String groupname
+      ) throws IOException {
+    if (username == null && groupname == null) {
+      throw new IOException("username == null && groupname == null");
+    }
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(p);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.setOwner(getPathName(p), username, groupname);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        fs.setOwner(p, username, groupname);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public void setTimes(Path p, final long mtime, final long atime
+      ) throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(p);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.setTimes(getPathName(p), mtime, atime);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        fs.setTimes(p, mtime, atime);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+  
+
+  @Override
+  protected int getDefaultPort() {
+    return HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
+  }
+
+  @Override
+  public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
+      throws IOException {
+    Token<DelegationTokenIdentifier> result =
+      dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
+    return result;
+  }
+
+  /**
+   * Requests the namenode to tell all datanodes to use a new, non-persistent
+   * bandwidth value for dfs.balance.bandwidthPerSec.
+   * The bandwidth parameter is the max number of bytes per second of network
+   * bandwidth to be used by a datanode during balancing.
+   *
+   * @param bandwidth Balancer bandwidth in bytes per second for all datanodes.
+   * @throws IOException
+   */
+  public void setBalancerBandwidth(long bandwidth) throws IOException {
+    dfs.setBalancerBandwidth(bandwidth);
+  }
+
+  /**
+   * Get a canonical service name for this file system. If the URI is logical,
+   * the hostname part of the URI will be returned.
+   * @return a service string that uniquely identifies this file system.
+   */
+  @Override
+  public String getCanonicalServiceName() {
+    return dfs.getCanonicalServiceName();
+  }
+  
+  @Override
+  protected URI canonicalizeUri(URI uri) {
+    if (HAUtilClient.isLogicalUri(getConf(), uri)) {
+      // Don't try to DNS-resolve logical URIs, since the 'authority'
+      // portion isn't a proper hostname
+      return uri;
+    } else {
+      return NetUtils.getCanonicalUri(uri, getDefaultPort());
+    }
+  }
+
+  /**
+   * Utility function that returns if the NameNode is in safemode or not. In HA
+   * mode, this API will return only ActiveNN's safemode status.
+   * 
+   * @return true if NameNode is in safemode, false otherwise.
+   * @throws IOException
+   *           when there is an issue communicating with the NameNode
+   */
+  public boolean isInSafeMode() throws IOException {
+    return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
+  }
+
+  /** @see HdfsAdmin#allowSnapshot(Path) */
+  public void allowSnapshot(final Path path) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.allowSnapshot(getPathName(p));
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          myDfs.allowSnapshot(p);
+        } else {
+          throw new UnsupportedOperationException("Cannot perform snapshot"
+              + " operations on a symlink to a non-DistributedFileSystem: "
+              + path + " -> " + p);
+        }
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+  
+  /** @see HdfsAdmin#disallowSnapshot(Path) */
+  public void disallowSnapshot(final Path path) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.disallowSnapshot(getPathName(p));
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          myDfs.disallowSnapshot(p);
+        } else {
+          throw new UnsupportedOperationException("Cannot perform snapshot"
+              + " operations on a symlink to a non-DistributedFileSystem: "
+              + path + " -> " + p);
+        }
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public Path createSnapshot(final Path path, final String snapshotName) 
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<Path>() {
+      @Override
+      public Path doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return new Path(dfs.createSnapshot(getPathName(p), snapshotName));
+      }
+
+      @Override
+      public Path next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          return myDfs.createSnapshot(p);
+        } else {
+          throw new UnsupportedOperationException("Cannot perform snapshot"
+              + " operations on a symlink to a non-DistributedFileSystem: "
+              + path + " -> " + p);
+        }
+      }
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public void renameSnapshot(final Path path, final String snapshotOldName,
+      final String snapshotNewName) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          myDfs.renameSnapshot(p, snapshotOldName, snapshotNewName);
+        } else {
+          throw new UnsupportedOperationException("Cannot perform snapshot"
+              + " operations on a symlink to a non-DistributedFileSystem: "
+              + path + " -> " + p);
+        }
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+  
+  /**
+   * @return All the snapshottable directories
+   * @throws IOException
+   */
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    return dfs.getSnapshottableDirListing();
+  }
+  
+  @Override
+  public void deleteSnapshot(final Path snapshotDir, final String snapshotName)
+      throws IOException {
+    Path absF = fixRelativePart(snapshotDir);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.deleteSnapshot(getPathName(p), snapshotName);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          myDfs.deleteSnapshot(p, snapshotName);
+        } else {
+          throw new UnsupportedOperationException("Cannot perform snapshot"
+              + " operations on a symlink to a non-DistributedFileSystem: "
+              + snapshotDir + " -> " + p);
+        }
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   * 
+   * @see DFSClient#getSnapshotDiffReport(String, String, String)
+   */
+  public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
+      final String fromSnapshot, final String toSnapshot) throws IOException {
+    Path absF = fixRelativePart(snapshotDir);
+    return new FileSystemLinkResolver<SnapshotDiffReport>() {
+      @Override
+      public SnapshotDiffReport doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
+            toSnapshot);
+      }
+
+      @Override
+      public SnapshotDiffReport next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          myDfs.getSnapshotDiffReport(p, fromSnapshot, toSnapshot);
+        } else {
+          throw new UnsupportedOperationException("Cannot perform snapshot"
+              + " operations on a symlink to a non-DistributedFileSystem: "
+              + snapshotDir + " -> " + p);
+        }
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+ 
+  /**
+   * Get the close status of a file
+   * @param src The path to the file
+   *
+   * @return return true if file is closed
+   * @throws FileNotFoundException if the file does not exist.
+   * @throws IOException If an I/O error occurred     
+   */
+  public boolean isFileClosed(final Path src) throws IOException {
+    Path absF = fixRelativePart(src);
+    return new FileSystemLinkResolver<Boolean>() {
+      @Override
+      public Boolean doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        return dfs.isFileClosed(getPathName(p));
+      }
+
+      @Override
+      public Boolean next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          return myDfs.isFileClosed(p);
+        } else {
+          throw new UnsupportedOperationException("Cannot call isFileClosed"
+              + " on a symlink to a non-DistributedFileSystem: "
+              + src + " -> " + p);
+        }
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * @see {@link #addCacheDirective(CacheDirectiveInfo, EnumSet)}
+   */
+  public long addCacheDirective(CacheDirectiveInfo info) throws IOException {
+    return addCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+  }
+
+  /**
+   * Add a new CacheDirective.
+   * 
+   * @param info Information about a directive to add.
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @return the ID of the directive that was created.
+   * @throws IOException if the directive could not be added
+   */
+  public long addCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    Preconditions.checkNotNull(info.getPath());
+    Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
+        makeQualified(getUri(), getWorkingDirectory());
+    return dfs.addCacheDirective(
+        new CacheDirectiveInfo.Builder(info).
+            setPath(path).
+            build(),
+        flags);
+  }
+
+  /**
+   * @see {@link #modifyCacheDirective(CacheDirectiveInfo, EnumSet)}
+   */
+  public void modifyCacheDirective(CacheDirectiveInfo info) throws IOException {
+    modifyCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+  }
+
+  /**
+   * Modify a CacheDirective.
+   * 
+   * @param info Information about the directive to modify. You must set the ID
+   *          to indicate which CacheDirective you want to modify.
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @throws IOException if the directive could not be modified
+   */
+  public void modifyCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    if (info.getPath() != null) {
+      info = new CacheDirectiveInfo.Builder(info).
+          setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
+              makeQualified(getUri(), getWorkingDirectory())).build();
+    }
+    dfs.modifyCacheDirective(info, flags);
+  }
+
+  /**
+   * Remove a CacheDirectiveInfo.
+   * 
+   * @param id identifier of the CacheDirectiveInfo to remove
+   * @throws IOException if the directive could not be removed
+   */
+  public void removeCacheDirective(long id)
+      throws IOException {
+    dfs.removeCacheDirective(id);
+  }
+  
+  /**
+   * List cache directives.  Incrementally fetches results from the server.
+   * 
+   * @param filter Filter parameters to use when listing the directives, null to
+   *               list all directives visible to us.
+   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
+   */
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      CacheDirectiveInfo filter) throws IOException {
+    if (filter == null) {
+      filter = new CacheDirectiveInfo.Builder().build();
+    }
+    if (filter.getPath() != null) {
+      filter = new CacheDirectiveInfo.Builder(filter).
+          setPath(new Path(getPathName(fixRelativePart(filter.getPath())))).
+          build();
+    }
+    final RemoteIterator<CacheDirectiveEntry> iter =
+        dfs.listCacheDirectives(filter);
+    return new RemoteIterator<CacheDirectiveEntry>() {
+      @Override
+      public boolean hasNext() throws IOException {
+        return iter.hasNext();
+      }
+
+      @Override
+      public CacheDirectiveEntry next() throws IOException {
+        // Although the paths we get back from the NameNode should always be
+        // absolute, we call makeQualified to add the scheme and authority of
+        // this DistributedFilesystem.
+        CacheDirectiveEntry desc = iter.next();
+        CacheDirectiveInfo info = desc.getInfo();
+        Path p = info.getPath().makeQualified(getUri(), getWorkingDirectory());
+        return new CacheDirectiveEntry(
+            new CacheDirectiveInfo.Builder(info).setPath(p).build(),
+            desc.getStats());
+      }
+    };
+  }
+
+  /**
+   * Add a cache pool.
+   *
+   * @param info
+   *          The request to add a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    CachePoolInfo.validate(info);
+    dfs.addCachePool(info);
+  }
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param info
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    CachePoolInfo.validate(info);
+    dfs.modifyCachePool(info);
+  }
+    
+  /**
+   * Remove a cache pool.
+   *
+   * @param poolName
+   *          Name of the cache pool to remove.
+   * @throws IOException 
+   *          if the cache pool did not exist, or could not be removed.
+   */
+  public void removeCachePool(String poolName) throws IOException {
+    CachePoolInfo.validateName(poolName);
+    dfs.removeCachePool(poolName);
+  }
+
+  /**
+   * List all cache pools.
+   *
+   * @return A remote iterator from which you can get CachePoolEntry objects.
+   *          Requests will be made as needed.
+   * @throws IOException
+   *          If there was an error listing cache pools.
+   */
+  public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+    return dfs.listCachePools();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void modifyAclEntries(Path path, final List<AclEntry> aclSpec)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.modifyAclEntries(getPathName(p), aclSpec);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        fs.modifyAclEntries(p, aclSpec);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void removeAclEntries(Path path, final List<AclEntry> aclSpec)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.removeAclEntries(getPathName(p), aclSpec);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        fs.removeAclEntries(p, aclSpec);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void removeDefaultAcl(Path path) throws IOException {
+    final Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.removeDefaultAcl(getPathName(p));
+        return null;
+      }
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        fs.removeDefaultAcl(p);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void removeAcl(Path path) throws IOException {
+    final Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.removeAcl(getPathName(p));
+        return null;
+      }
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        fs.removeAcl(p);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void setAcl(Path path, final List<AclEntry> aclSpec) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.setAcl(getPathName(p), aclSpec);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        fs.setAcl(p, aclSpec);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public AclStatus getAclStatus(Path path) throws IOException {
+    final Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<AclStatus>() {
+      @Override
+      public AclStatus doCall(final Path p) throws IOException {
+        return dfs.getAclStatus(getPathName(p));
+      }
+      @Override
+      public AclStatus next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        return fs.getAclStatus(p);
+      }
+    }.resolve(this, absF);
+  }
+  
+  /* HDFS only */
+  public void createEncryptionZone(final Path path, final String keyName)
+    throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        dfs.createEncryptionZone(getPathName(p), keyName);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          myDfs.createEncryptionZone(p, keyName);
+          return null;
+        } else {
+          throw new UnsupportedOperationException(
+              "Cannot call createEncryptionZone"
+                  + " on a symlink to a non-DistributedFileSystem: " + path
+                  + " -> " + p);
+        }
+      }
+    }.resolve(this, absF);
+  }
+
+  /* HDFS only */
+  public EncryptionZone getEZForPath(final Path path)
+          throws IOException {
+    Preconditions.checkNotNull(path);
+    Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<EncryptionZone>() {
+      @Override
+      public EncryptionZone doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        return dfs.getEZForPath(getPathName(p));
+      }
+
+      @Override
+      public EncryptionZone next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          return myDfs.getEZForPath(p);
+        } else {
+          throw new UnsupportedOperationException(
+              "Cannot call getEZForPath"
+                  + " on a symlink to a non-DistributedFileSystem: " + path
+                  + " -> " + p);
+        }
+      }
+    }.resolve(this, absF);
+  }
+
+  /* HDFS only */
+  public RemoteIterator<EncryptionZone> listEncryptionZones()
+      throws IOException {
+    return dfs.listEncryptionZones();
+  }
+
+  @Override
+  public void setXAttr(Path path, final String name, final byte[] value, 
+      final EnumSet<XAttrSetFlag> flag) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.setXAttr(getPathName(p), name, value, flag);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        fs.setXAttr(p, name, value, flag);
+        return null;
+      }      
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public byte[] getXAttr(Path path, final String name) throws IOException {
+    final Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<byte[]>() {
+      @Override
+      public byte[] doCall(final Path p) throws IOException {
+        return dfs.getXAttr(getPathName(p), name);
+      }
+      @Override
+      public byte[] next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        return fs.getXAttr(p, name);
+      }
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public Map<String, byte[]> getXAttrs(Path path) throws IOException {
+    final Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<Map<String, byte[]>>() {
+      @Override
+      public Map<String, byte[]> doCall(final Path p) throws IOException {
+        return dfs.getXAttrs(getPathName(p));
+      }
+      @Override
+      public Map<String, byte[]> next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        return fs.getXAttrs(p);
+      }
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public Map<String, byte[]> getXAttrs(Path path, final List<String> names) 
+      throws IOException {
+    final Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<Map<String, byte[]>>() {
+      @Override
+      public Map<String, byte[]> doCall(final Path p) throws IOException {
+        return dfs.getXAttrs(getPathName(p), names);
+      }
+      @Override
+      public Map<String, byte[]> next(final FileSystem fs, final Path p)
+        throws IOException, UnresolvedLinkException {
+        return fs.getXAttrs(p, names);
+      }
+    }.resolve(this, absF);
+  }
+  
+  @Override
+  public List<String> listXAttrs(Path path)
+          throws IOException {
+    final Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<List<String>>() {
+      @Override
+      public List<String> doCall(final Path p) throws IOException {
+        return dfs.listXAttrs(getPathName(p));
+      }
+      @Override
+      public List<String> next(final FileSystem fs, final Path p)
+              throws IOException, UnresolvedLinkException {
+        return fs.listXAttrs(p);
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public void removeXAttr(Path path, final String name) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.removeXAttr(getPathName(p), name);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        fs.removeXAttr(p, name);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public void access(Path path, final FsAction mode) throws IOException {
+    final Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.checkAccess(getPathName(p), mode);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        fs.access(p, mode);
+        return null;
+      }
+    }.resolve(this, absF);
+  }
+
+  @Override
+  public Token<?>[] addDelegationTokens(
+      final String renewer, Credentials credentials) throws IOException {
+    Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
+    if (dfs.isHDFSEncryptionEnabled()) {
+      KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension =
+          KeyProviderDelegationTokenExtension.
+              createKeyProviderDelegationTokenExtension(dfs.getKeyProvider());
+      Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
+          addDelegationTokens(renewer, credentials);
+      if (tokens != null && kpTokens != null) {
+        Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
+        System.arraycopy(tokens, 0, all, 0, tokens.length);
+        System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
+        tokens = all;
+      } else {
+        tokens = (tokens != null) ? tokens : kpTokens;
+      }
+    }
+    return tokens;
+  }
+
+  public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
+    return dfs.getInotifyEventStream();
+  }
+
+  public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
+      throws IOException {
+    return dfs.getInotifyEventStream(lastReadTxid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c030c6e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
new file mode 100644
index 0000000..77bed1a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
@@ -0,0 +1,105 @@
+/**
+ * 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.util.NoSuchElementException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Provides an iterator interface for listCorruptFileBlocks.
+ * This class is used by DistributedFileSystem and Hdfs.
+ */
+@InterfaceAudience.Private
+public class CorruptFileBlockIterator implements RemoteIterator<Path> {
+  private final DFSClient dfs;
+  private final String path;
+
+  private String[] files = null;
+  private int fileIdx = 0;
+  private String cookie = null;
+  private Path nextPath = null;
+
+  private int callsMade = 0;
+
+  public CorruptFileBlockIterator(DFSClient dfs, Path path) throws IOException {
+    this.dfs = dfs;
+    this.path = path2String(path);
+    loadNext();
+  }
+
+  /**
+   * @return the number of calls made to the DFSClient.
+   * This is for debugging and testing purposes.
+   */
+  public int getCallsMade() {
+    return callsMade;
+  }
+
+  private String path2String(Path path) {
+    return path.toUri().getPath();
+  }
+
+  private Path string2Path(String string) {
+    return new Path(string);
+  }
+
+  private void loadNext() throws IOException {
+    if (files == null || fileIdx >= files.length) {
+      CorruptFileBlocks cfb = dfs.listCorruptFileBlocks(path, cookie);
+      files = cfb.getFiles();
+      cookie = cfb.getCookie();
+      fileIdx = 0;
+      callsMade++;
+    }
+
+    if (fileIdx >= files.length) {
+      // received an empty response
+      // there are no more corrupt file blocks
+      nextPath = null;
+    } else {
+      nextPath = string2Path(files[fileIdx]);
+      fileIdx++;
+    }
+  }
+
+  
+  @Override
+  public boolean hasNext() {
+    return nextPath != null;
+  }
+
+  
+  @Override
+  public Path next() throws IOException {
+    if (!hasNext()) {
+      throw new NoSuchElementException("No more corrupt file blocks");
+    }
+
+    Path result = nextPath;
+    loadNext();
+
+    return result;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c030c6e/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 8c8afed..b934d2f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -983,6 +983,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9087. Add some jitter to DataNode.checkDiskErrorThread (Elliott Clark
     via Colin P. McCabe) 
 
+    HDFS-8740. Move DistributedFileSystem to hadoop-hdfs-client. (Mingliang Liu
+    via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[17/58] [abbrv] hadoop git commit: MAPREDUCE-6471. Document distcp incremental copy. Contributed by Neelesh Srinivas Salian.

Posted by zh...@apache.org.
MAPREDUCE-6471. Document distcp incremental copy. Contributed by Neelesh Srinivas Salian.


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

Branch: refs/heads/HDFS-7285
Commit: 66dad854c0aea8c137017fcf198b165cc1bd8bdd
Parents: 1c030c6
Author: Harsh J <ha...@cloudera.com>
Authored: Mon Sep 28 13:12:43 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Mon Sep 28 13:12:43 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                      | 3 +++
 hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm | 5 ++++-
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/66dad854/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index b7e9016..67adcbd 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -295,6 +295,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6471. Document distcp incremental copy
+    (Neelesh Srinivas Salian via harsh)
+
     MAPREDUCE-5045. UtilTest#isCygwin method appears to be unused
     (Neelesh Srinivas Salian via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66dad854/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
index 7424267..aacf4c7 100644
--- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
+++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
@@ -189,7 +189,9 @@ $H3 Update and Overwrite
   because it doesn't exist at the target. `10` and `20` are overwritten since
   the contents don't match the source.
 
-  If `-update` is used, `1` is overwritten as well.
+  If `-update` is used, `1` is skipped because the file-length and contents match. `2` is copied because it doesn’t exist at the target. `10` and `20` are overwritten since the contents don’t match the source. However, if `-append` is additionally used, then only `10` is overwritten (source length less than destination) and `20` is appended with the change in file (if the files match up to the destination's original length).
+
+  If `-overwrite` is used, `1` is overwritten as well.
 
 $H3 raw Namespace Extended Attribute Preservation
 
@@ -222,6 +224,7 @@ Flag              | Description                          | Notes
 `-m <num_maps>` | Maximum number of simultaneous copies | Specify the number of maps to copy data. Note that more maps may not necessarily improve throughput.
 `-overwrite` | Overwrite destination | If a map fails and `-i` is not specified, all the files in the split, not only those that failed, will be recopied. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.
 `-update` | Overwrite if source and destination differ in size, blocksize, or checksum | As noted in the preceding, this is not a "sync" operation. The criteria examined are the source and destination file sizes, blocksizes, and checksums; if they differ, the source file replaces the destination file. As discussed in the Usage documentation, it also changes the semantics for generating destination paths, so users should use this carefully.
+`-append` | Incremental copy of file with same name but different length | If the source file is greater in length than the destination file, the checksum of the common length part is compared. If the checksum matches, only the difference is copied using read and append functionalities. The -append option only works with `-update` without `-skipcrccheck`
 `-f <urilist_uri>` | Use list at \<urilist_uri\> as src list | This is equivalent to listing each source on the command line. The `urilist_uri` list should be a fully qualified URI.
 `-filelimit <n>` | Limit the total number of files to be <= n | **Deprecated!** Ignored in the new DistCp.
 `-sizelimit <n>` | Limit the total size to be <= n bytes | **Deprecated!** Ignored in the new DistCp.


[33/58] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 0000000,0000000..264c532
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@@ -1,0 -1,0 +1,952 @@@
++/**
++ * 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.util;
++
++import com.google.common.annotations.VisibleForTesting;
++
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.fs.StorageType;
++import org.apache.hadoop.hdfs.DFSClient;
++import org.apache.hadoop.hdfs.DFSStripedOutputStream;
++import org.apache.hadoop.hdfs.protocol.Block;
++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.LocatedStripedBlock;
++
++import com.google.common.base.Preconditions;
++import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
++import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
++import org.apache.hadoop.security.token.Token;
++
++import java.nio.ByteBuffer;
++import java.util.*;
++import java.io.IOException;
++import java.util.concurrent.CancellationException;
++import java.util.concurrent.CompletionService;
++import java.util.concurrent.ExecutionException;
++import java.util.concurrent.Future;
++import java.util.concurrent.TimeUnit;
++
++/**
++ * When accessing a file in striped layout, operations on logical byte ranges
++ * in the file need to be mapped to physical byte ranges on block files stored
++ * on DataNodes. This utility class facilities this mapping by defining and
++ * exposing a number of striping-related concepts. The most basic ones are
++ * illustrated in the following diagram. Unless otherwise specified, all
++ * range-related calculations are inclusive (the end offset of the previous
++ * range should be 1 byte lower than the start offset of the next one).
++ *
++ *  | <----  Block Group ----> |   <- Block Group: logical unit composing
++ *  |                          |        striped HDFS files.
++ *  blk_0      blk_1       blk_2   <- Internal Blocks: each internal block
++ *    |          |           |          represents a physically stored local
++ *    v          v           v          block file
++ * +------+   +------+   +------+
++ * |cell_0|   |cell_1|   |cell_2|  <- {@link StripingCell} represents the
++ * +------+   +------+   +------+       logical order that a Block Group should
++ * |cell_3|   |cell_4|   |cell_5|       be accessed: cell_0, cell_1, ...
++ * +------+   +------+   +------+
++ * |cell_6|   |cell_7|   |cell_8|
++ * +------+   +------+   +------+
++ * |cell_9|
++ * +------+  <- A cell contains cellSize bytes of data
++ */
++@InterfaceAudience.Private
++public class StripedBlockUtil {
++
++  /**
++   * This method parses a striped block group into individual blocks.
++   *
++   * @param bg The striped block group
++   * @param cellSize The size of a striping cell
++   * @param dataBlkNum The number of data blocks
++   * @return An array containing the blocks in the group
++   */
++  public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
++      int cellSize, int dataBlkNum, int parityBlkNum) {
++    int locatedBGSize = bg.getBlockIndices().length;
++    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum];
++    for (short i = 0; i < locatedBGSize; i++) {
++      final int idx = bg.getBlockIndices()[i];
++      // for now we do not use redundant replica of an internal block
++      if (idx < (dataBlkNum + parityBlkNum) && lbs[idx] == null) {
++        lbs[idx] = constructInternalBlock(bg, i, cellSize,
++            dataBlkNum, idx);
++      }
++    }
++    return lbs;
++  }
++
++  /**
++   * This method creates an internal block at the given index of a block group
++   *
++   * @param idxInReturnedLocs The index in the stored locations in the
++   *                          {@link LocatedStripedBlock} object
++   * @param idxInBlockGroup The logical index in the striped block group
++   * @return The constructed internal block
++   */
++  public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
++      int idxInReturnedLocs, int cellSize, int dataBlkNum,
++      int idxInBlockGroup) {
++    final ExtendedBlock blk = constructInternalBlock(
++        bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup);
++    final LocatedBlock locatedBlock;
++    if (idxInReturnedLocs < bg.getLocations().length) {
++      locatedBlock = new LocatedBlock(blk,
++          new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
++          new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
++          new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
++          bg.getStartOffset(), bg.isCorrupt(), null);
++    } else {
++      locatedBlock = new LocatedBlock(blk, null, null, null,
++          bg.getStartOffset(), bg.isCorrupt(), null);
++    }
++    Token<BlockTokenIdentifier>[] blockTokens = bg.getBlockTokens();
++    if (idxInReturnedLocs < blockTokens.length) {
++      locatedBlock.setBlockToken(blockTokens[idxInReturnedLocs]);
++    }
++    return locatedBlock;
++  }
++
++  /**
++   * This method creates an internal {@link ExtendedBlock} at the given index
++   * of a block group.
++   */
++  public static ExtendedBlock constructInternalBlock(ExtendedBlock blockGroup,
++      int cellSize, int dataBlkNum, int idxInBlockGroup) {
++    ExtendedBlock block = new ExtendedBlock(blockGroup);
++    block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup);
++    block.setNumBytes(getInternalBlockLength(blockGroup.getNumBytes(),
++        cellSize, dataBlkNum, idxInBlockGroup));
++    return block;
++  }
++
++  /**
++   * Get the size of an internal block at the given index of a block group
++   *
++   * @param dataSize Size of the block group only counting data blocks
++   * @param cellSize The size of a striping cell
++   * @param numDataBlocks The number of data blocks
++   * @param i The logical index in the striped block group
++   * @return The size of the internal block at the specified index
++   */
++  public static long getInternalBlockLength(long dataSize,
++      int cellSize, int numDataBlocks, int i) {
++    Preconditions.checkArgument(dataSize >= 0);
++    Preconditions.checkArgument(cellSize > 0);
++    Preconditions.checkArgument(numDataBlocks > 0);
++    Preconditions.checkArgument(i >= 0);
++    // Size of each stripe (only counting data blocks)
++    final int stripeSize = cellSize * numDataBlocks;
++    // If block group ends at stripe boundary, each internal block has an equal
++    // share of the group
++    final int lastStripeDataLen = (int)(dataSize % stripeSize);
++    if (lastStripeDataLen == 0) {
++      return dataSize / numDataBlocks;
++    }
++
++    final int numStripes = (int) ((dataSize - 1) / stripeSize + 1);
++    return (numStripes - 1L)*cellSize
++        + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
++  }
++
++  private static int lastCellSize(int size, int cellSize, int numDataBlocks,
++      int i) {
++    if (i < numDataBlocks) {
++      // parity block size (i.e. i >= numDataBlocks) is the same as 
++      // the first data block size (i.e. i = 0).
++      size -= i*cellSize;
++      if (size < 0) {
++        size = 0;
++      }
++    }
++    return size > cellSize? cellSize: size;
++  }
++
++  /**
++   * Given a byte's offset in an internal block, calculate the offset in
++   * the block group
++   */
++  public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum,
++      long offsetInBlk, int idxInBlockGroup) {
++    int cellIdxInBlk = (int) (offsetInBlk / cellSize);
++    return cellIdxInBlk * cellSize * dataBlkNum // n full stripes before offset
++        + idxInBlockGroup * cellSize // m full cells before offset
++        + offsetInBlk % cellSize; // partial cell
++  }
++
++  /**
++   * Get the next completed striped read task
++   *
++   * @return {@link StripingChunkReadResult} indicating the status of the read task
++   *          succeeded, and the block index of the task. If the method times
++   *          out without getting any completed read tasks, -1 is returned as
++   *          block index.
++   * @throws InterruptedException
++   */
++  public static StripingChunkReadResult getNextCompletedStripedRead(
++      CompletionService<Void> readService, Map<Future<Void>, Integer> futures,
++      final long timeoutMillis) throws InterruptedException {
++    Preconditions.checkArgument(!futures.isEmpty());
++    Future<Void> future = null;
++    try {
++      if (timeoutMillis > 0) {
++        future = readService.poll(timeoutMillis, TimeUnit.MILLISECONDS);
++      } else {
++        future = readService.take();
++      }
++      if (future != null) {
++        future.get();
++        return new StripingChunkReadResult(futures.remove(future),
++            StripingChunkReadResult.SUCCESSFUL);
++      } else {
++        return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT);
++      }
++    } catch (ExecutionException e) {
++      if (DFSClient.LOG.isDebugEnabled()) {
++        DFSClient.LOG.debug("ExecutionException " + e);
++      }
++      return new StripingChunkReadResult(futures.remove(future),
++          StripingChunkReadResult.FAILED);
++    } catch (CancellationException e) {
++      return new StripingChunkReadResult(futures.remove(future),
++          StripingChunkReadResult.CANCELLED);
++    }
++  }
++
++  /**
++   * Get the total usage of the striped blocks, which is the total of data
++   * blocks and parity blocks
++   *
++   * @param numDataBlkBytes
++   *          Size of the block group only counting data blocks
++   * @param dataBlkNum
++   *          The number of data blocks
++   * @param parityBlkNum
++   *          The number of parity blocks
++   * @param cellSize
++   *          The size of a striping cell
++   * @return The total usage of data blocks and parity blocks
++   */
++  public static long spaceConsumedByStripedBlock(long numDataBlkBytes,
++      int dataBlkNum, int parityBlkNum, int cellSize) {
++    int parityIndex = dataBlkNum + 1;
++    long numParityBlkBytes = getInternalBlockLength(numDataBlkBytes, cellSize,
++        dataBlkNum, parityIndex) * parityBlkNum;
++    return numDataBlkBytes + numParityBlkBytes;
++  }
++
++  /**
++   * Initialize the decoding input buffers based on the chunk states in an
++   * {@link AlignedStripe}. For each chunk that was not initially requested,
++   * schedule a new fetch request with the decoding input buffer as transfer
++   * destination.
++   */
++  public static byte[][] initDecodeInputs(AlignedStripe alignedStripe,
++      int dataBlkNum, int parityBlkNum) {
++    byte[][] decodeInputs =
++        new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()];
++    // read the full data aligned stripe
++    for (int i = 0; i < dataBlkNum; i++) {
++      if (alignedStripe.chunks[i] == null) {
++        final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
++        alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]);
++        alignedStripe.chunks[i].addByteArraySlice(0,
++            (int) alignedStripe.getSpanInBlock());
++      }
++    }
++    return decodeInputs;
++  }
++
++  /**
++   * Some fetched {@link StripingChunk} might be stored in original application
++   * buffer instead of prepared decode input buffers. Some others are beyond
++   * the range of the internal blocks and should correspond to all zero bytes.
++   * When all pending requests have returned, this method should be called to
++   * finalize decode input buffers.
++   */
++  public static void finalizeDecodeInputs(final byte[][] decodeInputs,
++      int dataBlkNum, int parityBlkNum, AlignedStripe alignedStripe) {
++    for (int i = 0; i < alignedStripe.chunks.length; i++) {
++      final StripingChunk chunk = alignedStripe.chunks[i];
++      final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
++      if (chunk != null && chunk.state == StripingChunk.FETCHED) {
++        chunk.copyTo(decodeInputs[decodeIndex]);
++      } else if (chunk != null && chunk.state == StripingChunk.ALLZERO) {
++        Arrays.fill(decodeInputs[decodeIndex], (byte) 0);
++      } else {
++        decodeInputs[decodeIndex] = null;
++      }
++    }
++  }
++
++  /**
++   * Currently decoding requires parity chunks are before data chunks.
++   * The indices are opposite to what we store in NN. In future we may
++   * improve the decoding to make the indices order the same as in NN.
++   *
++   * @param index The index to convert
++   * @param dataBlkNum The number of data blocks
++   * @param parityBlkNum The number of parity blocks
++   * @return converted index
++   */
++  public static int convertIndex4Decode(int index, int dataBlkNum,
++      int parityBlkNum) {
++    return index < dataBlkNum ? index + parityBlkNum : index - dataBlkNum;
++  }
++
++  public static int convertDecodeIndexBack(int index, int dataBlkNum,
++      int parityBlkNum) {
++    return index < parityBlkNum ? index + dataBlkNum : index - parityBlkNum;
++  }
++
++  /**
++   * Decode based on the given input buffers and erasure coding policy.
++   */
++  public static void decodeAndFillBuffer(final byte[][] decodeInputs,
++      AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum,
++      RawErasureDecoder decoder) {
++    // Step 1: prepare indices and output buffers for missing data units
++    int[] decodeIndices = new int[parityBlkNum];
++    int pos = 0;
++    for (int i = 0; i < dataBlkNum; i++) {
++      if (alignedStripe.chunks[i] != null &&
++          alignedStripe.chunks[i].state == StripingChunk.MISSING){
++        decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
++      }
++    }
++    decodeIndices = Arrays.copyOf(decodeIndices, pos);
++    byte[][] decodeOutputs =
++        new byte[decodeIndices.length][(int) alignedStripe.getSpanInBlock()];
++
++    // Step 2: decode into prepared output buffers
++    decoder.decode(decodeInputs, decodeIndices, decodeOutputs);
++
++    // Step 3: fill original application buffer with decoded data
++    for (int i = 0; i < decodeIndices.length; i++) {
++      int missingBlkIdx = convertDecodeIndexBack(decodeIndices[i],
++          dataBlkNum, parityBlkNum);
++      StripingChunk chunk = alignedStripe.chunks[missingBlkIdx];
++      if (chunk.state == StripingChunk.MISSING) {
++        chunk.copyFrom(decodeOutputs[i]);
++      }
++    }
++  }
++
++  /**
++   * Similar functionality with {@link #divideByteRangeIntoStripes}, but is used
++   * by stateful read and uses ByteBuffer as reading target buffer. Besides the
++   * read range is within a single stripe thus the calculation logic is simpler.
++   */
++  public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy,
++      int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup,
++      long rangeEndInBlockGroup, ByteBuffer buf) {
++    final int dataBlkNum = ecPolicy.getNumDataUnits();
++    // Step 1: map the byte range to StripingCells
++    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
++        blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
++
++    // Step 2: get the unmerged ranges on each internal block
++    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
++        cells);
++
++    // Step 3: merge into stripes
++    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
++
++    // Step 4: calculate each chunk's position in destination buffer. Since the
++    // whole read range is within a single stripe, the logic is simpler here.
++    int bufOffset = (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum));
++    for (StripingCell cell : cells) {
++      long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
++      long cellEnd = cellStart + cell.size - 1;
++      for (AlignedStripe s : stripes) {
++        long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
++        long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
++        long overlapEnd = Math.min(cellEnd, stripeEnd);
++        int overLapLen = (int) (overlapEnd - overlapStart + 1);
++        if (overLapLen > 0) {
++          Preconditions.checkState(s.chunks[cell.idxInStripe] == null);
++          final int pos = (int) (bufOffset + overlapStart - cellStart);
++          buf.position(pos);
++          buf.limit(pos + overLapLen);
++          s.chunks[cell.idxInStripe] = new StripingChunk(buf.slice());
++        }
++      }
++      bufOffset += cell.size;
++    }
++
++    // Step 5: prepare ALLZERO blocks
++    prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum);
++    return stripes;
++  }
++
++  /**
++   * This method divides a requested byte range into an array of inclusive
++   * {@link AlignedStripe}.
++   * @param ecPolicy The codec policy for the file, which carries the numbers
++   *                 of data / parity blocks
++   * @param cellSize Cell size of stripe
++   * @param blockGroup The striped block group
++   * @param rangeStartInBlockGroup The byte range's start offset in block group
++   * @param rangeEndInBlockGroup The byte range's end offset in block group
++   * @param buf Destination buffer of the read operation for the byte range
++   * @param offsetInBuf Start offset into the destination buffer
++   *
++   * At most 5 stripes will be generated from each logical range, as
++   * demonstrated in the header of {@link AlignedStripe}.
++   */
++  public static AlignedStripe[] divideByteRangeIntoStripes(ErasureCodingPolicy ecPolicy,
++      int cellSize, LocatedStripedBlock blockGroup,
++      long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
++      int offsetInBuf) {
++
++    // Step 0: analyze range and calculate basic parameters
++    final int dataBlkNum = ecPolicy.getNumDataUnits();
++
++    // Step 1: map the byte range to StripingCells
++    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
++        blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
++
++    // Step 2: get the unmerged ranges on each internal block
++    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
++        cells);
++
++    // Step 3: merge into at most 5 stripes
++    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
++
++    // Step 4: calculate each chunk's position in destination buffer
++    calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf);
++
++    // Step 5: prepare ALLZERO blocks
++    prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum);
++
++    return stripes;
++  }
++
++  /**
++   * Map the logical byte range to a set of inclusive {@link StripingCell}
++   * instances, each representing the overlap of the byte range to a cell
++   * used by {@link DFSStripedOutputStream} in encoding
++   */
++  @VisibleForTesting
++  private static StripingCell[] getStripingCellsOfByteRange(ErasureCodingPolicy ecPolicy,
++      int cellSize, LocatedStripedBlock blockGroup,
++      long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
++    Preconditions.checkArgument(
++        rangeStartInBlockGroup <= rangeEndInBlockGroup &&
++            rangeEndInBlockGroup < blockGroup.getBlockSize());
++    long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1;
++    int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
++    int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
++    int numCells = lastCellIdxInBG - firstCellIdxInBG + 1;
++    StripingCell[] cells = new StripingCell[numCells];
++
++    final int firstCellOffset = (int) (rangeStartInBlockGroup % cellSize);
++    final int firstCellSize =
++        (int) Math.min(cellSize - (rangeStartInBlockGroup % cellSize), len);
++    cells[0] = new StripingCell(ecPolicy, firstCellSize, firstCellIdxInBG,
++        firstCellOffset);
++    if (lastCellIdxInBG != firstCellIdxInBG) {
++      final int lastCellSize = (int) (rangeEndInBlockGroup % cellSize) + 1;
++      cells[numCells - 1] = new StripingCell(ecPolicy, lastCellSize,
++          lastCellIdxInBG, 0);
++    }
++
++    for (int i = 1; i < numCells - 1; i++) {
++      cells[i] = new StripingCell(ecPolicy, cellSize, i + firstCellIdxInBG, 0);
++    }
++
++    return cells;
++  }
++
++  /**
++   * Given a logical byte range, mapped to each {@link StripingCell}, calculate
++   * the physical byte range (inclusive) on each stored internal block.
++   */
++  @VisibleForTesting
++  private static VerticalRange[] getRangesForInternalBlocks(ErasureCodingPolicy ecPolicy,
++      int cellSize, StripingCell[] cells) {
++    int dataBlkNum = ecPolicy.getNumDataUnits();
++    int parityBlkNum = ecPolicy.getNumParityUnits();
++
++    VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum];
++
++    long earliestStart = Long.MAX_VALUE;
++    long latestEnd = -1;
++    for (StripingCell cell : cells) {
++      // iterate through all cells and update the list of StripeRanges
++      if (ranges[cell.idxInStripe] == null) {
++        ranges[cell.idxInStripe] = new VerticalRange(
++            cell.idxInInternalBlk * cellSize + cell.offset, cell.size);
++      } else {
++        ranges[cell.idxInStripe].spanInBlock += cell.size;
++      }
++      VerticalRange range = ranges[cell.idxInStripe];
++      if (range.offsetInBlock < earliestStart) {
++        earliestStart = range.offsetInBlock;
++      }
++      if (range.offsetInBlock + range.spanInBlock - 1 > latestEnd) {
++        latestEnd = range.offsetInBlock + range.spanInBlock - 1;
++      }
++    }
++
++    // Each parity block should be fetched at maximum range of all data blocks
++    for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
++      ranges[i] = new VerticalRange(earliestStart,
++          latestEnd - earliestStart + 1);
++    }
++
++    return ranges;
++  }
++
++  /**
++   * Merge byte ranges on each internal block into a set of inclusive
++   * {@link AlignedStripe} instances.
++   */
++  private static AlignedStripe[] mergeRangesForInternalBlocks(
++      ErasureCodingPolicy ecPolicy, VerticalRange[] ranges) {
++    int dataBlkNum = ecPolicy.getNumDataUnits();
++    int parityBlkNum = ecPolicy.getNumParityUnits();
++    List<AlignedStripe> stripes = new ArrayList<>();
++    SortedSet<Long> stripePoints = new TreeSet<>();
++    for (VerticalRange r : ranges) {
++      if (r != null) {
++        stripePoints.add(r.offsetInBlock);
++        stripePoints.add(r.offsetInBlock + r.spanInBlock);
++      }
++    }
++
++    long prev = -1;
++    for (long point : stripePoints) {
++      if (prev >= 0) {
++        stripes.add(new AlignedStripe(prev, point - prev,
++            dataBlkNum + parityBlkNum));
++      }
++      prev = point;
++    }
++    return stripes.toArray(new AlignedStripe[stripes.size()]);
++  }
++
++  private static void calcualteChunkPositionsInBuf(int cellSize,
++      AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
++      int offsetInBuf) {
++    /**
++     *     | <--------------- AlignedStripe --------------->|
++     *
++     *     |<- length_0 ->|<--  length_1  -->|<- length_2 ->|
++     * +------------------+------------------+----------------+
++     * |    cell_0_0_0    |    cell_3_1_0    |   cell_6_2_0   |  <- blk_0
++     * +------------------+------------------+----------------+
++     *   _/                \_______________________
++     *  |                                          |
++     *  v offset_0                                 v offset_1
++     * +----------------------------------------------------------+
++     * |  cell_0_0_0 |  cell_1_0_1 and cell_2_0_2  |cell_3_1_0 ...|   <- buf
++     * |  (partial)  |    (from blk_1 and blk_2)   |              |
++     * +----------------------------------------------------------+
++     *
++     * Cell indexing convention defined in {@link StripingCell}
++     */
++    int done = 0;
++    for (StripingCell cell : cells) {
++      long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
++      long cellEnd = cellStart + cell.size - 1;
++      for (AlignedStripe s : stripes) {
++        long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
++        long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
++        long overlapEnd = Math.min(cellEnd, stripeEnd);
++        int overLapLen = (int) (overlapEnd - overlapStart + 1);
++        if (overLapLen <= 0) {
++          continue;
++        }
++        if (s.chunks[cell.idxInStripe] == null) {
++          s.chunks[cell.idxInStripe] = new StripingChunk(buf);
++        }
++        s.chunks[cell.idxInStripe].addByteArraySlice(
++            (int)(offsetInBuf + done + overlapStart - cellStart), overLapLen);
++      }
++      done += cell.size;
++    }
++  }
++
++  /**
++   * If a {@link StripingChunk} maps to a byte range beyond an internal block's
++   * size, the chunk should be treated as zero bytes in decoding.
++   */
++  private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup,
++      AlignedStripe[] stripes, int cellSize, int dataBlkNum) {
++    for (AlignedStripe s : stripes) {
++      for (int i = 0; i < dataBlkNum; i++) {
++        long internalBlkLen = getInternalBlockLength(blockGroup.getBlockSize(),
++            cellSize, dataBlkNum, i);
++        if (internalBlkLen <= s.getOffsetInBlock()) {
++          Preconditions.checkState(s.chunks[i] == null);
++          s.chunks[i] = new StripingChunk(StripingChunk.ALLZERO);
++        }
++      }
++    }
++  }
++
++  /**
++   * Cell is the unit of encoding used in {@link DFSStripedOutputStream}. This
++   * size impacts how a logical offset in the file or block group translates
++   * to physical byte offset in a stored internal block. The StripingCell util
++   * class facilitates this calculation. Each StripingCell is inclusive with
++   * its start and end offsets -- e.g., the end logical offset of cell_0_0_0
++   * should be 1 byte lower than the start logical offset of cell_1_0_1.
++   *
++   *  | <------- Striped Block Group -------> |
++   *    blk_0          blk_1          blk_2
++   *      |              |              |
++   *      v              v              v
++   * +----------+   +----------+   +----------+
++   * |cell_0_0_0|   |cell_1_0_1|   |cell_2_0_2|
++   * +----------+   +----------+   +----------+
++   * |cell_3_1_0|   |cell_4_1_1|   |cell_5_1_2| <- {@link #idxInBlkGroup} = 5
++   * +----------+   +----------+   +----------+    {@link #idxInInternalBlk} = 1
++   *                                               {@link #idxInStripe} = 2
++   * A StripingCell is a special instance of {@link StripingChunk} whose offset
++   * and size align with the cell used when writing data.
++   * TODO: consider parity cells
++   */
++  @VisibleForTesting
++  static class StripingCell {
++    final ErasureCodingPolicy ecPolicy;
++    /** Logical order in a block group, used when doing I/O to a block group */
++    final int idxInBlkGroup;
++    final int idxInInternalBlk;
++    final int idxInStripe;
++    /**
++     * When a logical byte range is mapped to a set of cells, it might
++     * partially overlap with the first and last cells. This field and the
++     * {@link #size} variable represent the start offset and size of the
++     * overlap.
++     */
++    final int offset;
++    final int size;
++
++    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup,
++        int offset) {
++      this.ecPolicy = ecPolicy;
++      this.idxInBlkGroup = idxInBlkGroup;
++      this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits();
++      this.idxInStripe = idxInBlkGroup -
++          this.idxInInternalBlk * ecPolicy.getNumDataUnits();
++      this.offset = offset;
++      this.size = cellSize;
++    }
++  }
++
++  /**
++   * Given a requested byte range on a striped block group, an AlignedStripe
++   * represents an inclusive {@link VerticalRange} that is aligned with both
++   * the byte range and boundaries of all internal blocks. As illustrated in
++   * the diagram, any given byte range on a block group leads to 1~5
++   * AlignedStripe's.
++   *
++   * |<-------- Striped Block Group -------->|
++   * blk_0   blk_1   blk_2      blk_3   blk_4
++   *                 +----+  |  +----+  +----+
++   *                 |full|  |  |    |  |    | <- AlignedStripe0:
++   *         +----+  |~~~~|  |  |~~~~|  |~~~~|      1st cell is partial
++   *         |part|  |    |  |  |    |  |    | <- AlignedStripe1: byte range
++   * +----+  +----+  +----+  |  |~~~~|  |~~~~|      doesn't start at 1st block
++   * |full|  |full|  |full|  |  |    |  |    |
++   * |cell|  |cell|  |cell|  |  |    |  |    | <- AlignedStripe2 (full stripe)
++   * |    |  |    |  |    |  |  |    |  |    |
++   * +----+  +----+  +----+  |  |~~~~|  |~~~~|
++   * |full|  |part|          |  |    |  |    | <- AlignedStripe3: byte range
++   * |~~~~|  +----+          |  |~~~~|  |~~~~|      doesn't end at last block
++   * |    |                  |  |    |  |    | <- AlignedStripe4:
++   * +----+                  |  +----+  +----+      last cell is partial
++   *                         |
++   * <---- data blocks ----> | <--- parity --->
++   *
++   * An AlignedStripe is the basic unit of reading from a striped block group,
++   * because within the AlignedStripe, all internal blocks can be processed in
++   * a uniform manner.
++   *
++   * The coverage of an AlignedStripe on an internal block is represented as a
++   * {@link StripingChunk}.
++   *
++   * To simplify the logic of reading a logical byte range from a block group,
++   * a StripingChunk is either completely in the requested byte range or
++   * completely outside the requested byte range.
++   */
++  public static class AlignedStripe {
++    public VerticalRange range;
++    /** status of each chunk in the stripe */
++    public final StripingChunk[] chunks;
++    public int fetchedChunksNum = 0;
++    public int missingChunksNum = 0;
++
++    public AlignedStripe(long offsetInBlock, long length, int width) {
++      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
++      this.range = new VerticalRange(offsetInBlock, length);
++      this.chunks = new StripingChunk[width];
++    }
++
++    public boolean include(long pos) {
++      return range.include(pos);
++    }
++
++    public long getOffsetInBlock() {
++      return range.offsetInBlock;
++    }
++
++    public long getSpanInBlock() {
++      return range.spanInBlock;
++    }
++
++    @Override
++    public String toString() {
++      return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock +
++          ", fetchedChunksNum=" + fetchedChunksNum +
++          ", missingChunksNum=" + missingChunksNum;
++    }
++  }
++
++  /**
++   * A simple utility class representing an arbitrary vertical inclusive range
++   * starting at {@link #offsetInBlock} and lasting for {@link #spanInBlock}
++   * bytes in an internal block. Note that VerticalRange doesn't necessarily
++   * align with {@link StripingCell}.
++   *
++   * |<- Striped Block Group ->|
++   *  blk_0
++   *    |
++   *    v
++   * +-----+
++   * |~~~~~| <-- {@link #offsetInBlock}
++   * |     |  ^
++   * |     |  |
++   * |     |  | {@link #spanInBlock}
++   * |     |  v
++   * |~~~~~| ---
++   * |     |
++   * +-----+
++   */
++  public static class VerticalRange {
++    /** start offset in the block group (inclusive) */
++    public long offsetInBlock;
++    /** length of the stripe range */
++    public long spanInBlock;
++
++    public VerticalRange(long offsetInBlock, long length) {
++      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
++      this.offsetInBlock = offsetInBlock;
++      this.spanInBlock = length;
++    }
++
++    /** whether a position is in the range */
++    public boolean include(long pos) {
++      return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock;
++    }
++  }
++
++  /**
++   * Indicates the coverage of an {@link AlignedStripe} on an internal block,
++   * and the state of the chunk in the context of the read request.
++   *
++   * |<---------------- Striped Block Group --------------->|
++   *   blk_0        blk_1        blk_2          blk_3   blk_4
++   *                           +---------+  |  +----+  +----+
++   *     null         null     |REQUESTED|  |  |null|  |null| <- AlignedStripe0
++   *              +---------+  |---------|  |  |----|  |----|
++   *     null     |REQUESTED|  |REQUESTED|  |  |null|  |null| <- AlignedStripe1
++   * +---------+  +---------+  +---------+  |  +----+  +----+
++   * |REQUESTED|  |REQUESTED|    ALLZERO    |  |null|  |null| <- AlignedStripe2
++   * +---------+  +---------+               |  +----+  +----+
++   * <----------- data blocks ------------> | <--- parity --->
++   */
++  public static class StripingChunk {
++    /** Chunk has been successfully fetched */
++    public static final int FETCHED = 0x01;
++    /** Chunk has encountered failed when being fetched */
++    public static final int MISSING = 0x02;
++    /** Chunk being fetched (fetching task is in-flight) */
++    public static final int PENDING = 0x04;
++    /**
++     * Chunk is requested either by application or for decoding, need to
++     * schedule read task
++     */
++    public static final int REQUESTED = 0X08;
++    /**
++     * Internal block is short and has no overlap with chunk. Chunk considered
++     * all-zero bytes in codec calculations.
++     */
++    public static final int ALLZERO = 0X0f;
++
++    /**
++     * If a chunk is completely in requested range, the state transition is:
++     * REQUESTED (when AlignedStripe created) -> PENDING -> {FETCHED | MISSING}
++     * If a chunk is completely outside requested range (including parity
++     * chunks), state transition is:
++     * null (AlignedStripe created) -> REQUESTED (upon failure) -> PENDING ...
++     */
++    public int state = REQUESTED;
++
++    public final ChunkByteArray byteArray;
++    public final ByteBuffer byteBuffer;
++
++    public StripingChunk(byte[] buf) {
++      this.byteArray = new ChunkByteArray(buf);
++      byteBuffer = null;
++    }
++
++    public StripingChunk(ByteBuffer buf) {
++      this.byteArray = null;
++      this.byteBuffer = buf;
++    }
++
++    public StripingChunk(int state) {
++      this.byteArray = null;
++      this.byteBuffer = null;
++      this.state = state;
++    }
++
++    public void addByteArraySlice(int offset, int length) {
++      assert byteArray != null;
++      byteArray.offsetsInBuf.add(offset);
++      byteArray.lengthsInBuf.add(length);
++    }
++
++    void copyTo(byte[] target) {
++      assert byteArray != null;
++      byteArray.copyTo(target);
++    }
++
++    void copyFrom(byte[] src) {
++      assert byteArray != null;
++      byteArray.copyFrom(src);
++    }
++  }
++
++  public static class ChunkByteArray {
++    private final byte[] buf;
++    private final List<Integer> offsetsInBuf;
++    private final List<Integer> lengthsInBuf;
++
++    ChunkByteArray(byte[] buf) {
++      this.buf = buf;
++      this.offsetsInBuf = new ArrayList<>();
++      this.lengthsInBuf = new ArrayList<>();
++    }
++
++    public int[] getOffsets() {
++      int[] offsets = new int[offsetsInBuf.size()];
++      for (int i = 0; i < offsets.length; i++) {
++        offsets[i] = offsetsInBuf.get(i);
++      }
++      return offsets;
++    }
++
++    public int[] getLengths() {
++      int[] lens = new int[this.lengthsInBuf.size()];
++      for (int i = 0; i < lens.length; i++) {
++        lens[i] = this.lengthsInBuf.get(i);
++      }
++      return lens;
++    }
++
++    public byte[] buf() {
++      return buf;
++    }
++
++    void copyTo(byte[] target) {
++      int posInBuf = 0;
++      for (int i = 0; i < offsetsInBuf.size(); i++) {
++        System.arraycopy(buf, offsetsInBuf.get(i),
++            target, posInBuf, lengthsInBuf.get(i));
++        posInBuf += lengthsInBuf.get(i);
++      }
++    }
++
++    void copyFrom(byte[] src) {
++      int srcPos = 0;
++      for (int j = 0; j < offsetsInBuf.size(); j++) {
++        System.arraycopy(src, srcPos, buf, offsetsInBuf.get(j),
++            lengthsInBuf.get(j));
++        srcPos += lengthsInBuf.get(j);
++      }
++    }
++  }
++
++  /**
++   * This class represents result from a striped read request.
++   * If the task was successful or the internal computation failed,
++   * an index is also returned.
++   */
++  public static class StripingChunkReadResult {
++    public static final int SUCCESSFUL = 0x01;
++    public static final int FAILED = 0x02;
++    public static final int TIMEOUT = 0x04;
++    public static final int CANCELLED = 0x08;
++
++    public final int index;
++    public final int state;
++
++    public StripingChunkReadResult(int state) {
++      Preconditions.checkArgument(state == TIMEOUT,
++          "Only timeout result should return negative index.");
++      this.index = -1;
++      this.state = state;
++    }
++
++    public StripingChunkReadResult(int index, int state) {
++      Preconditions.checkArgument(state != TIMEOUT,
++          "Timeout result should return negative index.");
++      this.index = index;
++      this.state = state;
++    }
++
++    @Override
++    public String toString() {
++      return "(index=" + index + ", state =" + state + ")";
++    }
++  }
++
++  /**
++   * Check if the information such as IDs and generation stamps in block-i
++   * match the block group.
++   */
++  public static void checkBlocks(ExtendedBlock blockGroup,
++      int i, ExtendedBlock blocki) throws IOException {
++    if (!blocki.getBlockPoolId().equals(blockGroup.getBlockPoolId())) {
++      throw new IOException("Block pool IDs mismatched: block" + i + "="
++          + blocki + ", expected block group=" + blockGroup);
++    }
++    if (blocki.getBlockId() - i != blockGroup.getBlockId()) {
++      throw new IOException("Block IDs mismatched: block" + i + "="
++          + blocki + ", expected block group=" + blockGroup);
++    }
++    if (blocki.getGenerationStamp() != blockGroup.getGenerationStamp()) {
++      throw new IOException("Generation stamps mismatched: block" + i + "="
++          + blocki + ", expected block group=" + blockGroup);
++    }
++  }
++
++  public static int getBlockIndex(Block reportedBlock) {
++    long BLOCK_GROUP_INDEX_MASK = 15;
++    return (int) (reportedBlock.getBlockId() &
++        BLOCK_GROUP_INDEX_MASK);
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index b28ab42,0e2d541..d35fb57
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@@ -648,3 -444,3 +478,11 @@@ message RollingUpgradeStatusProto 
    required string blockPoolId = 1;
    optional bool finalized = 2 [default = false];
  }
++
++
++/**
++ * A list of storage IDs.
++ */
++message StorageUuidsProto {
++  repeated string storageUuids = 1;
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 8874c4d,b631955..0166029
--- 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
@@@ -402,14 -397,11 +400,19 @@@ public class DFSConfigKeys extends Comm
    public static final int     DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT = 21600;
    public static final String  DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY = "dfs.datanode.directoryscan.threads";
    public static final int     DFS_DATANODE_DIRECTORYSCAN_THREADS_DEFAULT = 1;
 +  public static final String  DFS_DATANODE_STRIPED_READ_THREADS_KEY = "dfs.datanode.stripedread.threads";
 +  public static final int     DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20;
 +  public static final String  DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size";
 +  public static final int     DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024;
 +  public static final String  DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY = "dfs.datanode.stripedread.timeout.millis";
 +  public static final int     DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT = 5000; //5s
 +  public static final String  DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size";
 +  public static final int     DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8;
+   public static final String
+       DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY =
+       "dfs.datanode.directoryscan.throttle.limit.ms.per.sec";
+   public static final int
+       DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT = 1000;
    public static final String  DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface";
    public static final String  DFS_DATANODE_DNS_INTERFACE_DEFAULT = "default";
    public static final String  DFS_DATANODE_DNS_NAMESERVER_KEY = "dfs.datanode.dns.nameserver";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 7f721f0,5b11ac2..b0ea7ce
--- 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
@@@ -1442,34 -1439,5 +1441,4 @@@ public class DFSUtil 
          .createKeyProviderCryptoExtension(keyProvider);
      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();
-   }
--
-   public static InterruptedIOException toInterruptedIOException(String message,
-       InterruptedException e) {
-     final InterruptedIOException iioe = new InterruptedIOException(message);
-     iioe.initCause(e);
-     return iioe;
-   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 524248c,75b3811..05c498f
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@@ -47,36 -45,27 +47,35 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockKeyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockWithLocationsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlocksWithLocationsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointCommandProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointSignatureProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ExportedBlockKeysProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamespaceInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RecoveringBlockProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogManifestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ReplicaStateProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageInfoProto;
 -import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageUuidsProto;
  import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
  import org.apache.hadoop.hdfs.security.token.block.BlockKey;
  import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 9228bec,2646089..43ddf74
--- 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
@@@ -364,10 -368,9 +370,9 @@@ public class DataNode extends Reconfigu
    private String supergroup;
    private boolean isPermissionEnabled;
    private String dnUserName = null;
- 
-   private SpanReceiverHost spanReceiverHost;
--
 +  private ErasureCodingWorker ecWorker;
+   final Tracer tracer;
+   private final TracerConfigurationManager tracerConfigurationManager;
    private static final int NUM_CORES = Runtime.getRuntime()
        .availableProcessors();
    private static final double CONGESTION_RATIO = 1.5;
@@@ -3289,12 -3287,8 +3320,12 @@@
    @Override
    public void removeSpanReceiver(long id) throws IOException {
      checkSuperuserPrivilege();
-     spanReceiverHost.removeSpanReceiver(id);
+     tracerConfigurationManager.removeSpanReceiver(id);
    }
 +  
 +  public ErasureCodingWorker getErasureCodingWorker(){
 +    return ecWorker;
 +  }
  
    /**
     * Get timeout value of each OOB type from configuration


[26/58] [abbrv] hadoop git commit: HADOOP-12446. Undeprecate createNonRecursive(). Contributed by Ted Yu.

Posted by zh...@apache.org.
HADOOP-12446. Undeprecate createNonRecursive(). Contributed by Ted Yu.


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

Branch: refs/heads/HDFS-7285
Commit: 3abbdc929bde05f8819f5410cef1eaeb8940203f
Parents: ab11085
Author: Kihwal Lee <ki...@apache.org>
Authored: Mon Sep 28 17:31:51 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Mon Sep 28 17:31:51 2015 -0500

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                | 2 ++
 .../src/main/java/org/apache/hadoop/fs/FileSystem.java         | 6 ------
 .../src/main/java/org/apache/hadoop/fs/FilterFileSystem.java   | 1 -
 .../src/main/java/org/apache/hadoop/fs/HarFileSystem.java      | 1 -
 .../src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java | 1 -
 .../java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java   | 1 -
 .../src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java  | 1 -
 .../java/org/apache/hadoop/hdfs/DistributedFileSystem.java     | 1 -
 .../java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java     | 1 -
 .../src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java | 1 -
 .../java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java | 3 ---
 .../apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java  | 1 -
 12 files changed, 2 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index c7c5de2..07463f4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -787,6 +787,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12428. Fix inconsistency between log-level guards and statements.
     (Jagadesh Kiran N and Jackie Chang via ozawa)
 
+    HADOOP-12446. Undeprecate createNonRecursive() (Ted Yu via kihwal)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 2bcaa54..6f13a87 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1089,9 +1089,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    * @param progress
    * @throws IOException
    * @see #setPermission(Path, FsPermission)
-   * @deprecated API only for 0.20-append
    */
-  @Deprecated
   public FSDataOutputStream createNonRecursive(Path f,
       boolean overwrite,
       int bufferSize, short replication, long blockSize,
@@ -1114,9 +1112,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    * @param progress
    * @throws IOException
    * @see #setPermission(Path, FsPermission)
-   * @deprecated API only for 0.20-append
    */
-   @Deprecated
    public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
        boolean overwrite, int bufferSize, short replication, long blockSize,
        Progressable progress) throws IOException {
@@ -1139,9 +1135,7 @@ public abstract class FileSystem extends Configured implements Closeable {
     * @param progress
     * @throws IOException
     * @see #setPermission(Path, FsPermission)
-    * @deprecated API only for 0.20-append
     */
-    @Deprecated
     public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
         EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
         Progressable progress) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
index 815ef69..f862c74 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
@@ -203,7 +203,6 @@ public class FilterFileSystem extends FileSystem {
 
 
   @Override
-  @Deprecated
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
index 714702b..868b8dc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
@@ -713,7 +713,6 @@ public class HarFileSystem extends FileSystem {
     throw new IOException("Har: create not allowed.");
   }
 
-  @SuppressWarnings("deprecation")
   @Override
   public FSDataOutputStream createNonRecursive(Path f, boolean overwrite,
       int bufferSize, short replication, long blockSize, Progressable progress)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index 8ff65fa..352b27a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -319,7 +319,6 @@ public class RawLocalFileSystem extends FileSystem {
   }
   
   @Override
-  @Deprecated
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
index f7a93e7..dd28c58 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
@@ -185,7 +185,6 @@ class ChRootedFileSystem extends FilterFileSystem {
   }
   
   @Override
-  @Deprecated
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
index 1710ba4..b179c36 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
@@ -55,7 +55,6 @@ public class TestHarFileSystem {
    * {@link HarFileSystem}. Either because there is a default implementation
    * already available or because it is not relevant.
    */
-  @SuppressWarnings("deprecation")
   private interface MustNotImplement {
     public BlockLocation[] getFileBlockLocations(Path p, long start, long len);
     public long getLength(Path f);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 88e6637..ea86d2d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -446,7 +446,6 @@ public class DistributedFileSystem extends FileSystem {
    * Same as create(), except fails if parent directory doesn't already exist.
    */
   @Override
-  @SuppressWarnings("deprecation")
   public FSDataOutputStream createNonRecursive(final Path f,
       final FsPermission permission, final EnumSet<CreateFlag> flag,
       final int bufferSize, final short replication, final long blockSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index cfda3fe..e245d2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -1179,7 +1179,6 @@ public class WebHdfsFileSystem extends FileSystem
   }
 
   @Override
-  @SuppressWarnings("deprecation")
   public FSDataOutputStream createNonRecursive(final Path f,
       final FsPermission permission, final EnumSet<CreateFlag> flag,
       final int bufferSize, final short replication, final long blockSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index e59963b..4ad066f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -860,7 +860,6 @@ public class TestFileCreation {
   // Attempts to create and close a file using FileSystem.createNonRecursive(),
   // catching and returning an exception if one occurs or null
   // if the operation is successful.
-  @SuppressWarnings("deprecation")
   static IOException createNonRecursive(FileSystem fs, Path name,
       int repl, EnumSet<CreateFlag> flag) throws IOException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index bb9941b..910582e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -1204,7 +1204,6 @@ public class NativeAzureFileSystem extends FileSystem {
   }
 
   @Override
-  @SuppressWarnings("deprecation")
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
@@ -1279,7 +1278,6 @@ public class NativeAzureFileSystem extends FileSystem {
   }
 
   @Override
-  @SuppressWarnings("deprecation")
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
@@ -1298,7 +1296,6 @@ public class NativeAzureFileSystem extends FileSystem {
   }
 
   @Override
-  @SuppressWarnings("deprecation")
   public FSDataOutputStream createNonRecursive(Path f,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3abbdc92/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
index 6989a70..1f07677 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
@@ -1332,7 +1332,6 @@ public abstract class NativeAzureFileSystemBaseTest {
     return (lastModified > (time - errorMargin) && lastModified < (time + errorMargin));
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   public void testCreateNonRecursive() throws Exception {
     Path testFolder = new Path("/testFolder");


[14/58] [abbrv] hadoop git commit: HADOOP-11984. Enable parallel JUnit tests in pre-commit (Contributed by Chris Nauroth)

Posted by zh...@apache.org.
HADOOP-11984. Enable parallel JUnit tests in pre-commit (Contributed by Chris Nauroth)


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

Branch: refs/heads/HDFS-7285
Commit: f0f984e4e63d0dbafe93062a122ee051330db301
Parents: 0b31c23
Author: Vinayakumar B <vi...@apache.org>
Authored: Sun Sep 27 14:12:07 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Sun Sep 27 14:12:07 2015 +0530

----------------------------------------------------------------------
 dev-support/test-patch.sh                       | 29 +++++++++-
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 hadoop-common-project/hadoop-common/pom.xml     | 43 +++++++++++++-
 .../apache/hadoop/conf/TestConfiguration.java   |  2 +-
 .../apache/hadoop/ha/ClientBaseWithFixes.java   | 40 ++-----------
 .../http/TestAuthenticationSessionCookie.java   |  5 +-
 .../apache/hadoop/http/TestHttpCookieFlag.java  |  7 +--
 .../apache/hadoop/http/TestSSLHttpServer.java   |  4 +-
 .../org/apache/hadoop/net/ServerSocketUtil.java |  7 ++-
 .../hadoop/security/ssl/KeyStoreTestUtil.java   | 60 ++++++++++++++++++--
 hadoop-project/pom.xml                          |  2 +-
 11 files changed, 142 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index a3cdc85..23b2272 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -599,6 +599,8 @@ function hadoop_usage
   echo "--run-tests            Run all relevant tests below the base directory"
   echo "--skip-system-plugins  Do not load plugins from ${BINDIR}/test-patch.d"
   echo "--testlist=<list>      Specify which subsystem tests to use (comma delimited)"
+  echo "--test-parallel=<bool> Run multiple tests in parallel (default false in developer mode, true in Jenkins mode)"
+  echo "--test-threads=<int>   Number of tests to run in parallel (default defined in ${PROJECT_NAME} build)"
 
   echo "Shell binary overrides:"
   echo "--awk-cmd=<cmd>        The 'awk' command to use (default 'awk')"
@@ -691,6 +693,7 @@ function parse_args
       ;;
       --jenkins)
         JENKINS=true
+        TEST_PARALLEL=${TEST_PARALLEL:-true}
       ;;
       --jira-cmd=*)
         JIRACLI=${i#*=}
@@ -749,6 +752,12 @@ function parse_args
           add_test "${j}"
         done
       ;;
+      --test-parallel=*)
+        TEST_PARALLEL=${i#*=}
+      ;;
+      --test-threads=*)
+        TEST_THREADS=${i#*=}
+      ;;
       --wget-cmd=*)
         WGET=${i#*=}
       ;;
@@ -811,6 +820,13 @@ function parse_args
   PATCH_DIR=$(cd -P -- "${PATCH_DIR}" >/dev/null && pwd -P)
 
   GITDIFFLINES=${PATCH_DIR}/gitdifflines.txt
+
+  if [[ ${TEST_PARALLEL} == "true" ]] ; then
+    PARALLEL_TESTS_PROFILE=-Pparallel-tests
+    if [[ -n ${TEST_THREADS:-} ]]; then
+      TESTS_THREAD_COUNT="-DtestsThreadCount=$TEST_THREADS"
+    fi
+  fi
 }
 
 ## @description  Locate the pom.xml file for a given directory
@@ -2245,13 +2261,22 @@ function check_unittests
 
     test_logfile=${PATCH_DIR}/testrun_${module_suffix}.txt
     echo "  Running tests in ${module_suffix}"
-    echo_and_redirect "${test_logfile}" "${MVN}" "${MAVEN_ARGS[@]}" clean install -fae ${NATIVE_PROFILE} ${REQUIRE_TEST_LIB_HADOOP} -D${PROJECT_NAME}PatchProcess
+    # Temporary hack to run the parallel tests profile only for hadoop-common.
+    # This code will be removed once hadoop-hdfs is ready for parallel test
+    # execution.
+    if [[ ${module} == "hadoop-common-project/hadoop-common" ]] ; then
+      OPTIONAL_PARALLEL_TESTS_PROFILE=${PARALLEL_TESTS_PROFILE}
+    else
+      unset OPTIONAL_PARALLEL_TESTS_PROFILE
+    fi
+    # shellcheck disable=2086
+    echo_and_redirect "${test_logfile}" "${MVN}" "${MAVEN_ARGS[@]}" clean install -fae ${NATIVE_PROFILE} ${REQUIRE_TEST_LIB_HADOOP} ${OPTIONAL_PARALLEL_TESTS_PROFILE} ${TESTS_THREAD_COUNT} -D${PROJECT_NAME}PatchProcess
     test_build_result=$?
 
     add_jira_footer "${module_suffix} test log" "@@BASE@@/testrun_${module_suffix}.txt"
 
     # shellcheck disable=2016
-    module_test_timeouts=$(${AWK} '/^Running / { if (last) { print last } last=$2 } /^Tests run: / { last="" }' "${test_logfile}")
+    module_test_timeouts=$(${AWK} '/^Running / { array[$NF] = 1 } /^Tests run: .* in / { delete array[$NF] } END { for (x in array) { print x } }' "${test_logfile}")
     if [[ -n "${module_test_timeouts}" ]] ; then
       test_timeouts="${test_timeouts} ${module_test_timeouts}"
       result=1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 77929dc..c7c5de2 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -822,6 +822,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11878. FileContext#fixRelativePart should check for not null for a
     more informative exception. (Brahma Reddy Battula via kasha)
 
+    HADOOP-11984. Enable parallel JUnit tests in pre-commit.
+    (Chris Nauroth via vinayakumarb)
+
   BUG FIXES
 
     HADOOP-12374. Updated expunge command description.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 3ae09a0..ef77dbd 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -879,11 +879,52 @@
       <build>
         <plugins>
           <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-parallel-tests-dirs</id>
+                <phase>test-compile</phase>
+                <configuration>
+                  <target>
+                    <script language="javascript"><![CDATA[
+                      var baseDirs = [
+                          "${test.build.data}",
+                          "${test.build.dir}",
+                          "${hadoop.tmp.dir}" ];
+                      for (var i in baseDirs) {
+                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
+                          var mkdir = project.createTask("mkdir");
+                          mkdir.setDir(new java.io.File(baseDirs[i], j));
+                          mkdir.perform();
+                        }
+                      }
+                    ]]></script>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-surefire-plugin</artifactId>
             <configuration>
               <forkCount>${testsThreadCount}</forkCount>
-              <argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -DminiClusterDedicatedDirs=true</argLine>
+              <reuseForks>false</reuseForks>
+              <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+              <systemPropertyVariables>
+                <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+
+                <!-- Due to a Maven quirk, setting this to just -->
+                <!-- surefire.forkNumber won't do the parameter substitution. -->
+                <!-- Putting a prefix in front of it like "fork-" makes it -->
+                <!-- work. -->
+                <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+              </systemPropertyVariables>
             </configuration>
           </plugin>
         </plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index a039741..46b2e50 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -1493,7 +1493,7 @@ public class TestConfiguration extends TestCase {
 
       @Override
       public void run() {
-        for (int i = 0; i < 100000; i++) {
+        for (int i = 0; i < 10000; i++) {
           config.set("some.config.value-" + prefix + i, "value");
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
index 5f03133..b1ce1d1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
@@ -23,10 +23,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
-import java.io.RandomAccessFile;
 import java.net.Socket;
-import java.nio.channels.FileLock;
-import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
@@ -34,8 +31,8 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.util.Time;
-import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -167,10 +164,6 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
     private LinkedList<ZooKeeper> allClients;
     private boolean allClientsSetup = false;
 
-    private RandomAccessFile portNumLockFile;
-
-    private File portNumFile;
-
     protected TestableZooKeeper createClient(CountdownWatcher watcher, String hp)
         throws IOException, InterruptedException
     {
@@ -413,29 +406,11 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
 
     private String initHostPort() {
         BASETEST.mkdirs();
-        int port;
-        for (;;) {
-            port = PortAssignment.unique();
-            FileLock lock = null;
-            portNumLockFile = null;
-            try {
-                try {
-                    portNumFile = new File(BASETEST, port + ".lock");
-                    portNumLockFile = new RandomAccessFile(portNumFile, "rw");
-                    try {
-                        lock = portNumLockFile.getChannel().tryLock();
-                    } catch (OverlappingFileLockException e) {
-                        continue;
-                    }
-                } finally {
-                    if (lock != null)
-                        break;
-                    if (portNumLockFile != null)
-                        portNumLockFile.close();
-                }
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+        int port = 0;
+        try {
+           port = ServerSocketUtil.getPort(port, 100);
+        } catch (IOException e) {
+           throw new RuntimeException(e);
         }
         return "127.0.0.1:" + port;
     }
@@ -480,9 +455,6 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
 
         stopServer();
 
-        portNumLockFile.close();
-        portNumFile.delete();
-        
         if (tmpDir != null) {
             Assert.assertTrue("delete " + tmpDir.toString(), recursiveDelete(tmpDir));
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
index e435034..058633a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
@@ -113,10 +113,7 @@ public class TestAuthenticationSessionCookie {
     sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
 
     KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
-    Configuration sslConf = new Configuration(false);
-    sslConf.addResource("ssl-server.xml");
-    sslConf.addResource("ssl-client.xml");
-
+    Configuration sslConf = KeyStoreTestUtil.getSslConfig();
 
     server = new HttpServer2.Builder()
             .setName("test")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
index 5c5ed48..f73b019 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
@@ -17,7 +17,6 @@ import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.ssl.SSLFactory;
@@ -27,12 +26,10 @@ import org.junit.Test;
 
 import javax.net.ssl.HttpsURLConnection;
 import javax.servlet.*;
-import javax.servlet.http.Cookie;
 import javax.servlet.http.HttpServletResponse;
 import java.io.File;
 import java.io.IOException;
 import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
 import java.security.GeneralSecurityException;
@@ -89,9 +86,7 @@ public class TestHttpCookieFlag {
     sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
 
     KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
-    Configuration sslConf = new Configuration(false);
-    sslConf.addResource("ssl-server.xml");
-    sslConf.addResource("ssl-client.xml");
+    Configuration sslConf = KeyStoreTestUtil.getSslConfig();
 
     clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, sslConf);
     clientSslFactory.init();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
index 70fea87..1371964 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
@@ -65,9 +65,7 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
     sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
 
     KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
-    Configuration sslConf = new Configuration(false);
-    sslConf.addResource("ssl-server.xml");
-    sslConf.addResource("ssl-client.xml");
+    Configuration sslConf = KeyStoreTestUtil.getSslConfig();
 
     clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, sslConf);
     clientSslFactory.init();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
index 0ce835f..1917287 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 public class ServerSocketUtil {
 
   private static final Log LOG = LogFactory.getLog(ServerSocketUtil.class);
+  private static Random rand = new Random();
 
   /**
    * Port scan & allocate is how most other apps find ports
@@ -38,13 +39,15 @@ public class ServerSocketUtil {
    * @throws IOException
    */
   public static int getPort(int port, int retries) throws IOException {
-    Random rand = new Random();
     int tryPort = port;
     int tries = 0;
     while (true) {
-      if (tries > 0) {
+      if (tries > 0 || tryPort == 0) {
         tryPort = port + rand.nextInt(65535 - port);
       }
+      if (tryPort == 0) {
+        continue;
+      }
       LOG.info("Using port " + tryPort);
       try (ServerSocket s = new ServerSocket(tryPort)) {
         return tryPort;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
index 07cae8b..453ae48 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
@@ -37,7 +37,6 @@ import java.security.KeyPair;
 import java.security.KeyPairGenerator;
 import java.security.KeyStore;
 import java.security.NoSuchAlgorithmException;
-import java.security.PrivateKey;
 import java.security.SecureRandom;
 import java.security.cert.Certificate;
 import java.security.cert.X509Certificate;
@@ -49,8 +48,6 @@ import java.security.InvalidKeyException;
 import java.security.NoSuchProviderException;
 import java.security.SignatureException;
 import java.security.cert.CertificateEncodingException;
-import java.security.cert.CertificateException;
-import java.security.cert.CertificateFactory;
 import javax.security.auth.x500.X500Principal;
 import org.bouncycastle.x509.X509V1CertificateGenerator;
 
@@ -233,8 +230,8 @@ public class KeyStoreTestUtil {
     String trustKS = null;
     String trustPassword = "trustP";
 
-    File sslClientConfFile = new File(sslConfDir + "/ssl-client.xml");
-    File sslServerConfFile = new File(sslConfDir + "/ssl-server.xml");
+    File sslClientConfFile = new File(sslConfDir, getClientSSLConfigFileName());
+    File sslServerConfFile = new File(sslConfDir, getServerSSLConfigFileName());
 
     Map<String, X509Certificate> certs = new HashMap<String, X509Certificate>();
 
@@ -312,8 +309,44 @@ public class KeyStoreTestUtil {
   }
 
   /**
+   * Returns the client SSL configuration file name.  Under parallel test
+   * execution, this file name is parameterized by a unique ID to ensure that
+   * concurrent tests don't collide on an SSL configuration file.
+   *
+   * @return client SSL configuration file name
+   */
+  public static String getClientSSLConfigFileName() {
+    return getSSLConfigFileName("ssl-client");
+  }
+
+  /**
+   * Returns the server SSL configuration file name.  Under parallel test
+   * execution, this file name is parameterized by a unique ID to ensure that
+   * concurrent tests don't collide on an SSL configuration file.
+   *
+   * @return client SSL configuration file name
+   */
+  public static String getServerSSLConfigFileName() {
+    return getSSLConfigFileName("ssl-server");
+  }
+
+  /**
+   * Returns an SSL configuration file name.  Under parallel test
+   * execution, this file name is parameterized by a unique ID to ensure that
+   * concurrent tests don't collide on an SSL configuration file.
+   *
+   * @param base the base of the file name
+   * @return SSL configuration file name for base
+   */
+  private static String getSSLConfigFileName(String base) {
+    String testUniqueForkId = System.getProperty("test.unique.fork.id");
+    String fileSuffix = testUniqueForkId != null ? "-" + testUniqueForkId : "";
+    return base + fileSuffix + ".xml";
+  }
+
+  /**
    * Creates SSL configuration.
-   * 
+   *
    * @param mode SSLFactory.Mode mode to configure
    * @param keystore String keystore file
    * @param password String store password, or null to avoid setting store
@@ -410,4 +443,19 @@ public class KeyStoreTestUtil {
       throw e;
     }
   }
+
+  /**
+   * Get the SSL configuration
+   * @return {@link Configuration} instance with ssl configs loaded
+   */
+  public static Configuration getSslConfig(){
+    Configuration sslConf = new Configuration(false);
+    String sslServerConfFile = KeyStoreTestUtil.getServerSSLConfigFileName();
+    String sslClientConfFile = KeyStoreTestUtil.getClientSSLConfigFileName();
+    sslConf.addResource(sslServerConfFile);
+    sslConf.addResource(sslClientConfFile);
+    sslConf.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile);
+    sslConf.set(SSLFactory.SSL_CLIENT_CONF_KEY, sslClientConfFile);
+    return sslConf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 636e063..293c279 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -90,7 +90,7 @@
     <enforced.maven.version>[3.0.2,)</enforced.maven.version>
 
     <!-- Plugin versions and config -->
-    <maven-surefire-plugin.argLine>-Xmx4096m -XX:MaxPermSize=768m -XX:+HeapDumpOnOutOfMemoryError</maven-surefire-plugin.argLine>
+    <maven-surefire-plugin.argLine>-Xmx2048m -XX:MaxPermSize=768m -XX:+HeapDumpOnOutOfMemoryError</maven-surefire-plugin.argLine>
     <maven-surefire-plugin.version>2.17</maven-surefire-plugin.version>
     <maven-surefire-report-plugin.version>${maven-surefire-plugin.version}</maven-surefire-report-plugin.version>
     <maven-failsafe-plugin.version>${maven-surefire-plugin.version}</maven-failsafe-plugin.version>


[56/58] [abbrv] hadoop git commit: HADOOP-12448. TestTextCommand: use mkdirs rather than mkdir to create test directory. (Contributed by Colin Patrick McCabe and Chris Nauroth)

Posted by zh...@apache.org.
HADOOP-12448. TestTextCommand: use mkdirs rather than mkdir to create test directory. (Contributed by Colin Patrick McCabe and Chris Nauroth)


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

Branch: refs/heads/HDFS-7285
Commit: 06abc57a90bb2ac86121cc803a60dab82609da88
Parents: 071733d
Author: yliu <yl...@apache.org>
Authored: Wed Sep 30 10:56:22 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Wed Sep 30 10:56:22 2015 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt      |  3 +++
 .../org/apache/hadoop/fs/shell/TestTextCommand.java  | 15 +++++++++------
 2 files changed, 12 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06abc57a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index ec7d1c6..84535d6 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1100,6 +1100,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-12447. Clean up some htrace integration issues (cmccabe)
 
+    HADOOP-12448. TestTextCommand: use mkdirs rather than mkdir to create test
+    directory. (Contributed by Colin Patrick McCabe and Chris Nauroth)
+
   OPTIMIZATIONS
 
     HADOOP-12051. ProtobufRpcEngine.invoke() should use Exception.toString()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06abc57a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java
index 70a2f03..0e33d6a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java
@@ -22,11 +22,13 @@ import static org.junit.Assert.*;
 
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.InputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.StringWriter;
 import java.lang.reflect.Method;
 import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -38,12 +40,13 @@ import org.junit.Test;
  * by the Text command.
  */
 public class TestTextCommand {
-  private static final String TEST_ROOT_DIR =
-    System.getProperty("test.build.data", "build/test/data/") + "/testText";
+  private static final File TEST_ROOT_DIR =
+    Paths.get(System.getProperty("test.build.data", "build/test/data"),
+        "testText").toFile();
   private static final String AVRO_FILENAME =
-    new Path(TEST_ROOT_DIR, "weather.avro").toUri().getPath();
+    new File(TEST_ROOT_DIR, "weather.avro").toURI().getPath();
   private static final String TEXT_FILENAME =
-    new Path(TEST_ROOT_DIR, "testtextfile.txt").toUri().getPath();
+    new File(TEST_ROOT_DIR, "testtextfile.txt").toURI().getPath();
 
   /**
    * Tests whether binary Avro data files are displayed correctly.
@@ -126,7 +129,7 @@ public class TestTextCommand {
   }
 
   private void createFile(String fileName, byte[] contents) throws IOException {
-    (new File(TEST_ROOT_DIR)).mkdir();
+    Files.createDirectories(TEST_ROOT_DIR.toPath());
     File file = new File(fileName);
     file.createNewFile();
     FileOutputStream stream = new FileOutputStream(file);


[57/58] [abbrv] hadoop git commit: YARN-3727. For better error recovery, check if the directory exists before using it for localization. Contributed by Zhihai Xu

Posted by zh...@apache.org.
YARN-3727. For better error recovery, check if the directory exists before using it for localization. Contributed by Zhihai Xu


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

Branch: refs/heads/HDFS-7285
Commit: 854d25b0c30fd40f640c052e79a8747741492042
Parents: 06abc57
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Sep 30 14:59:44 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Sep 30 14:59:44 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../localizer/LocalResourcesTracker.java        |  3 +-
 .../localizer/LocalResourcesTrackerImpl.java    | 24 ++++++--
 .../localizer/ResourceLocalizationService.java  |  5 +-
 .../TestLocalResourcesTrackerImpl.java          | 65 +++++++++++++++++---
 .../TestResourceLocalizationService.java        | 16 ++---
 6 files changed, 94 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/854d25b0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 43c501f..bf3ac12 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1005,6 +1005,9 @@ Release 2.7.2 - UNRELEASED
     YARN-4180. AMLauncher does not retry on failures when talking to NM.
     (adhoot)
 
+    YARN-3727. For better error recovery, check if the directory exists before
+    using it for localization. (Zhihai Xu via jlowe)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/854d25b0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java
index 14ec911..56e3de5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java
@@ -33,7 +33,8 @@ interface LocalResourcesTracker
 
   boolean remove(LocalizedResource req, DeletionService delService);
 
-  Path getPathForLocalization(LocalResourceRequest req, Path localDirPath);
+  Path getPathForLocalization(LocalResourceRequest req, Path localDirPath,
+      DeletionService delService);
 
   String getUser();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/854d25b0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
index a1e6817..51dbcaa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
@@ -440,10 +440,12 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
    * @param {@link LocalResourceRequest} Resource localization request to
    *        localize the resource.
    * @param {@link Path} local directory path
+   * @param {@link DeletionService} Deletion Service to delete existing
+   *        path for localization.
    */
   @Override
-  public Path
-      getPathForLocalization(LocalResourceRequest req, Path localDirPath) {
+  public Path getPathForLocalization(LocalResourceRequest req,
+      Path localDirPath, DeletionService delService) {
     Path rPath = localDirPath;
     if (useLocalCacheDirectoryManager && localDirPath != null) {
 
@@ -463,8 +465,22 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
       inProgressLocalResourcesMap.put(req, rPath);
     }
 
-    rPath = new Path(rPath,
-        Long.toString(uniqueNumberGenerator.incrementAndGet()));
+    while (true) {
+      Path uniquePath = new Path(rPath,
+          Long.toString(uniqueNumberGenerator.incrementAndGet()));
+      File file = new File(uniquePath.toUri().getRawPath());
+      if (!file.exists()) {
+        rPath = uniquePath;
+        break;
+      }
+      // If the directory already exists, delete it and move to next one.
+      LOG.warn("Directory " + uniquePath + " already exists, " +
+          "try next one.");
+      if (delService != null) {
+        delService.delete(getUser(), uniquePath);
+      }
+    }
+
     Path localPath = new Path(rPath, req.getPath().getName());
     LocalizedResource rsrc = localrsrc.get(req);
     rsrc.setLocalPath(localPath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/854d25b0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index e239e34..2cc5683 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -830,7 +830,8 @@ public class ResourceLocalizationService extends CompositeService
                     + ContainerLocalizer.FILECACHE,
                   ContainerLocalizer.getEstimatedSize(resource), true);
             Path publicDirDestPath =
-                publicRsrc.getPathForLocalization(key, publicRootPath);
+                publicRsrc.getPathForLocalization(key, publicRootPath,
+                    delService);
             if (!publicDirDestPath.getParent().equals(publicRootPath)) {
               DiskChecker.checkDir(new File(publicDirDestPath.toUri().getPath()));
             }
@@ -1116,7 +1117,7 @@ public class ResourceLocalizationService extends CompositeService
           dirsHandler.getLocalPathForWrite(cacheDirectory,
             ContainerLocalizer.getEstimatedSize(rsrc), false);
       return tracker.getPathForLocalization(new LocalResourceRequest(rsrc),
-          dirPath);
+          dirPath, delService);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/854d25b0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java
index 350cecb..e6aeae0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java
@@ -35,6 +35,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -427,7 +428,7 @@ public class TestLocalResourcesTrackerImpl {
       // Simulate the process of localization of lr1
       // NOTE: Localization path from tracker has resource ID at end
       Path hierarchicalPath1 =
-          tracker.getPathForLocalization(lr1, localDir).getParent();
+          tracker.getPathForLocalization(lr1, localDir, null).getParent();
       // Simulate lr1 getting localized
       ResourceLocalizedEvent rle1 =
           new ResourceLocalizedEvent(lr1,
@@ -444,7 +445,7 @@ public class TestLocalResourcesTrackerImpl {
       tracker.handle(reqEvent2);
 
       Path hierarchicalPath2 =
-          tracker.getPathForLocalization(lr2, localDir).getParent();
+          tracker.getPathForLocalization(lr2, localDir, null).getParent();
       // localization failed.
       ResourceFailedLocalizationEvent rfe2 =
           new ResourceFailedLocalizationEvent(
@@ -463,7 +464,7 @@ public class TestLocalResourcesTrackerImpl {
           LocalResourceVisibility.PUBLIC, lc1);
       tracker.handle(reqEvent3);
       Path hierarchicalPath3 =
-          tracker.getPathForLocalization(lr3, localDir).getParent();
+          tracker.getPathForLocalization(lr3, localDir, null).getParent();
       // localization successful
       ResourceLocalizedEvent rle3 =
           new ResourceLocalizedEvent(lr3, new Path(hierarchicalPath3.toUri()
@@ -542,7 +543,8 @@ public class TestLocalResourcesTrackerImpl {
       dispatcher.await();
 
       // Simulate the process of localization of lr1
-      Path hierarchicalPath1 = tracker.getPathForLocalization(lr1, localDir);
+      Path hierarchicalPath1 = tracker.getPathForLocalization(lr1, localDir,
+          null);
 
       ArgumentCaptor<LocalResourceProto> localResourceCaptor =
           ArgumentCaptor.forClass(LocalResourceProto.class);
@@ -622,7 +624,8 @@ public class TestLocalResourcesTrackerImpl {
       dispatcher.await();
 
       // Simulate the process of localization of lr1
-      Path hierarchicalPath1 = tracker.getPathForLocalization(lr1, localDir);
+      Path hierarchicalPath1 = tracker.getPathForLocalization(lr1, localDir,
+          null);
 
       ArgumentCaptor<LocalResourceProto> localResourceCaptor =
           ArgumentCaptor.forClass(LocalResourceProto.class);
@@ -691,7 +694,8 @@ public class TestLocalResourcesTrackerImpl {
           LocalResourceVisibility.APPLICATION, lc2);
       tracker.handle(reqEvent2);
       dispatcher.await();
-      Path hierarchicalPath2 = tracker.getPathForLocalization(lr2, localDir);
+      Path hierarchicalPath2 = tracker.getPathForLocalization(lr2, localDir,
+          null);
       long localizedId2 = Long.parseLong(hierarchicalPath2.getName());
       Assert.assertEquals(localizedId1 + 1, localizedId2);
     } finally {
@@ -785,6 +789,49 @@ public class TestLocalResourcesTrackerImpl {
     }
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testGetPathForLocalization() throws Exception {
+    FileContext lfs = FileContext.getLocalFSFileContext();
+    Path base_path = new Path("target",
+        TestLocalResourcesTrackerImpl.class.getSimpleName());
+    final String user = "someuser";
+    final ApplicationId appId = ApplicationId.newInstance(1, 1);
+    Configuration conf = new YarnConfiguration();
+    DrainDispatcher dispatcher = null;
+    dispatcher = createDispatcher(conf);
+    EventHandler<LocalizerEvent> localizerEventHandler =
+        mock(EventHandler.class);
+    EventHandler<LocalizerEvent> containerEventHandler =
+        mock(EventHandler.class);
+    dispatcher.register(LocalizerEventType.class, localizerEventHandler);
+    dispatcher.register(ContainerEventType.class, containerEventHandler);
+    NMStateStoreService stateStore = mock(NMStateStoreService.class);
+    DeletionService delService = mock(DeletionService.class);
+    try {
+      LocalResourceRequest req1 = createLocalResourceRequest(user, 1, 1,
+          LocalResourceVisibility.PUBLIC);
+      LocalizedResource lr1 = createLocalizedResource(req1, dispatcher);
+      ConcurrentMap<LocalResourceRequest, LocalizedResource> localrsrc =
+          new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+      localrsrc.put(req1, lr1);
+      LocalResourcesTrackerImpl tracker = new LocalResourcesTrackerImpl(user,
+          appId, dispatcher, localrsrc, true, conf, stateStore, null);
+      Path conflictPath = new Path(base_path, "10");
+      Path qualifiedConflictPath = lfs.makeQualified(conflictPath);
+      lfs.mkdir(qualifiedConflictPath, null, true);
+      Path rPath = tracker.getPathForLocalization(req1, base_path,
+          delService);
+      Assert.assertFalse(lfs.util().exists(rPath));
+      verify(delService, times(1)).delete(eq(user), eq(conflictPath));
+    } finally {
+      lfs.delete(base_path, true);
+      if (dispatcher != null) {
+        dispatcher.stop();
+      }
+    }
+  }
+
   @SuppressWarnings("unchecked")
   @Test
   public void testResourcePresentInGoodDir() throws IOException {
@@ -832,8 +879,10 @@ public class TestLocalResourcesTrackerImpl {
       tracker.handle(req21Event);
       dispatcher.await();
       // Localize resource1
-      Path p1 = tracker.getPathForLocalization(req1, new Path("/tmp/somedir1"));
-      Path p2 = tracker.getPathForLocalization(req2, new Path("/tmp/somedir2"));
+      Path p1 = tracker.getPathForLocalization(req1,
+          new Path("/tmp/somedir1"), null);
+      Path p2 = tracker.getPathForLocalization(req2,
+          new Path("/tmp/somedir2"), null);
       ResourceLocalizedEvent rle1 = new ResourceLocalizedEvent(req1, p1, 1);
       tracker.handle(rle1);
       ResourceLocalizedEvent rle2 = new ResourceLocalizedEvent(req2, p2, 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/854d25b0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index c515506..e7a9db8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -624,29 +624,31 @@ public class TestResourceLocalizationService {
       // Simulate start of localization for all resources
       privTracker1.getPathForLocalization(privReq1,
           dirsHandler.getLocalPathForWrite(
-              ContainerLocalizer.USERCACHE + user1));
+              ContainerLocalizer.USERCACHE + user1), null);
       privTracker1.getPathForLocalization(privReq2,
           dirsHandler.getLocalPathForWrite(
-              ContainerLocalizer.USERCACHE + user1));
+              ContainerLocalizer.USERCACHE + user1), null);
       LocalizedResource privLr1 = privTracker1.getLocalizedResource(privReq1);
       LocalizedResource privLr2 = privTracker1.getLocalizedResource(privReq2);
       appTracker1.getPathForLocalization(appReq1,
           dirsHandler.getLocalPathForWrite(
-              ContainerLocalizer.APPCACHE + appId1));
+              ContainerLocalizer.APPCACHE + appId1), null);
       LocalizedResource appLr1 = appTracker1.getLocalizedResource(appReq1);
       appTracker2.getPathForLocalization(appReq2,
           dirsHandler.getLocalPathForWrite(
-              ContainerLocalizer.APPCACHE + appId2));
+              ContainerLocalizer.APPCACHE + appId2), null);
       LocalizedResource appLr2 = appTracker2.getLocalizedResource(appReq2);
       appTracker2.getPathForLocalization(appReq3,
           dirsHandler.getLocalPathForWrite(
-              ContainerLocalizer.APPCACHE + appId2));
+              ContainerLocalizer.APPCACHE + appId2), null);
       LocalizedResource appLr3 = appTracker2.getLocalizedResource(appReq3);
       pubTracker.getPathForLocalization(pubReq1,
-          dirsHandler.getLocalPathForWrite(ContainerLocalizer.FILECACHE));
+          dirsHandler.getLocalPathForWrite(ContainerLocalizer.FILECACHE),
+          null);
       LocalizedResource pubLr1 = pubTracker.getLocalizedResource(pubReq1);
       pubTracker.getPathForLocalization(pubReq2,
-          dirsHandler.getLocalPathForWrite(ContainerLocalizer.FILECACHE));
+          dirsHandler.getLocalPathForWrite(ContainerLocalizer.FILECACHE),
+          null);
       LocalizedResource pubLr2 = pubTracker.getLocalizedResource(pubReq2);
 
       // Simulate completion of localization for most resources with


[20/58] [abbrv] hadoop git commit: HDFS-9080. Update htrace version to 4.0.1 (cmccabe)

Posted by zh...@apache.org.
HDFS-9080. Update htrace version to 4.0.1 (cmccabe)


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

Branch: refs/heads/HDFS-7285
Commit: 892ade689f9bcce76daae8f66fc00a49bee8548e
Parents: 66dad85
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Sat Sep 26 22:05:51 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Sep 28 07:42:12 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/pom.xml     |   2 +-
 .../hadoop/fs/CommonConfigurationKeys.java      |   3 +
 .../org/apache/hadoop/fs/FSOutputSummer.java    |  12 +-
 .../java/org/apache/hadoop/fs/FileContext.java  |   7 +
 .../java/org/apache/hadoop/fs/FileSystem.java   |  21 +-
 .../main/java/org/apache/hadoop/fs/FsShell.java |  25 +--
 .../java/org/apache/hadoop/fs/FsTracer.java     |  64 ++++++
 .../main/java/org/apache/hadoop/fs/Globber.java |  15 +-
 .../main/java/org/apache/hadoop/ipc/Client.java |  13 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |  17 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |  51 +++--
 .../apache/hadoop/ipc/WritableRpcEngine.java    |  13 +-
 .../apache/hadoop/tracing/SpanReceiverHost.java | 208 -------------------
 .../org/apache/hadoop/tracing/TraceUtils.java   |  24 ++-
 .../tracing/TracerConfigurationManager.java     | 100 +++++++++
 .../java/org/apache/hadoop/util/ProtoUtil.java  |  13 +-
 .../src/main/proto/RpcHeader.proto              |   5 +-
 .../src/main/resources/core-default.xml         |  15 ++
 .../hadoop-common/src/site/markdown/Tracing.md  |  80 +++----
 .../java/org/apache/hadoop/fs/TestFsShell.java  |  11 +-
 .../apache/hadoop/tracing/SetSpanReceiver.java  |  13 +-
 .../apache/hadoop/tracing/TestTraceUtils.java   |   2 +-
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  18 +-
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  21 +-
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  30 +--
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 201 +++++++++---------
 .../hadoop/hdfs/DFSInotifyEventInputStream.java |  29 ++-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  22 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  21 +-
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  31 +--
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  71 ++++---
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |  20 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  23 +-
 .../hdfs/protocol/CacheDirectiveIterator.java   |  13 +-
 .../hadoop/hdfs/protocol/CachePoolIterator.java |  13 +-
 .../hdfs/protocol/EncryptionZoneIterator.java   |  14 +-
 .../datatransfer/DataTransferProtoUtil.java     |  42 +---
 .../hdfs/protocol/datatransfer/Sender.java      |  22 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   2 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |  35 +++-
 .../hdfs/qjournal/server/JournalNode.java       |  13 +-
 .../qjournal/server/JournalNodeRpcServer.java   |   1 +
 .../hdfs/server/datanode/BlockSender.java       |   9 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  35 +++-
 .../hdfs/server/datanode/DataXceiver.java       |   2 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  25 ++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  10 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   5 +
 .../src/main/resources/hdfs-default.xml         |  13 +-
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |   2 +
 .../hadoop/hdfs/TestBlockReaderLocal.java       |   2 +
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |   5 +-
 .../org/apache/hadoop/hdfs/TestDFSPacket.java   |  25 +--
 .../server/namenode/TestCacheDirectives.java    |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   3 +-
 .../hadoop/tools/TestHdfsConfigFields.java      |   6 +-
 .../apache/hadoop/tracing/TestTraceAdmin.java   |   9 +-
 .../org/apache/hadoop/tracing/TestTracing.java  |  84 +++++---
 .../TestTracingShortCircuitLocalRead.java       |  18 +-
 hadoop-project/pom.xml                          |   4 +-
 62 files changed, 857 insertions(+), 770 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index ef77dbd..21af670 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -246,7 +246,7 @@
 
     <dependency>
       <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core</artifactId>
+      <artifactId>htrace-core4</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.zookeeper</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 2721466..09d2550 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -295,4 +295,7 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String NFS_EXPORTS_ALLOWED_HOSTS_SEPARATOR = ";";
   public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY = "nfs.exports.allowed.hosts";
   public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT = "* rw";
+
+  // HDFS client HTrace configuration.
+  public static final String  FS_CLIENT_HTRACE_PREFIX = "fs.client.htrace.";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index bdc5585..648043e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@ -21,8 +21,8 @@ package org.apache.hadoop.fs;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.NullScope;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -43,6 +43,8 @@ abstract public class FSOutputSummer extends OutputStream {
   private byte checksum[];
   // The number of valid bytes in the buffer.
   private int count;
+  // The HTrace tracer to use
+  private Tracer tracer;
   
   // We want this value to be a multiple of 3 because the native code checksums
   // 3 chunks simultaneously. The chosen value of 9 strikes a balance between
@@ -197,7 +199,7 @@ abstract public class FSOutputSummer extends OutputStream {
   }
 
   protected TraceScope createWriteTraceScope() {
-    return NullScope.INSTANCE;
+    return null;
   }
 
   /** Generate checksums for the given data chunks and output chunks & checksums
@@ -215,7 +217,9 @@ abstract public class FSOutputSummer extends OutputStream {
             getChecksumSize());
       }
     } finally {
-      scope.close();
+      if (scope != null) {
+        scope.close();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index a98d662..4dbf9e3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ShutdownHookManager;
 
 import com.google.common.base.Preconditions;
+import org.apache.htrace.core.Tracer;
 
 /**
  * The FileContext class provides an interface for users of the Hadoop
@@ -222,12 +223,14 @@ public class FileContext {
   private final Configuration conf;
   private final UserGroupInformation ugi;
   final boolean resolveSymlinks;
+  private final Tracer tracer;
 
   private FileContext(final AbstractFileSystem defFs,
     final FsPermission theUmask, final Configuration aConf) {
     defaultFS = defFs;
     umask = FsPermission.getUMask(aConf);
     conf = aConf;
+    tracer = FsTracer.get(aConf);
     try {
       ugi = UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
@@ -2721,4 +2724,8 @@ public class FileContext {
       throws IOException {
     return defaultFS.getAllStoragePolicies();
   }
+
+  Tracer getTracer() {
+    return tracer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 8f32644..2bcaa54 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -67,9 +67,8 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.Tracer;
+import org.apache.htrace.core.TraceScope;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -129,6 +128,13 @@ public abstract class FileSystem extends Configured implements Closeable {
   private Set<Path> deleteOnExit = new TreeSet<Path>();
   
   boolean resolveSymlinks;
+
+  private Tracer tracer;
+
+  protected final Tracer getTracer() {
+    return tracer;
+  }
+
   /**
    * This method adds a file system for testing so that we can find it later. It
    * is only for testing.
@@ -2706,14 +2712,13 @@ public abstract class FileSystem extends Configured implements Closeable {
 
   private static FileSystem createFileSystem(URI uri, Configuration conf
       ) throws IOException {
-    TraceScope scope = Trace.startSpan("FileSystem#createFileSystem");
-    Span span = scope.getSpan();
-    if (span != null) {
-      span.addKVAnnotation("scheme", uri.getScheme());
-    }
+    Tracer tracer = FsTracer.get(conf);
+    TraceScope scope = tracer.newScope("FileSystem#createFileSystem");
+    scope.addKVAnnotation("scheme", uri.getScheme());
     try {
       Class<?> clazz = getFileSystemClass(uri.getScheme(), conf);
       FileSystem fs = (FileSystem)ReflectionUtils.newInstance(clazz, conf);
+      fs.tracer = tracer;
       fs.initialize(uri, conf);
       return fs;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
index 35608e2..d91866e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
@@ -32,16 +32,13 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFactory;
 import org.apache.hadoop.fs.shell.FsCommand;
-import org.apache.hadoop.tracing.SpanReceiverHost;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.tracing.TraceUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.SamplerBuilder;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 /** Provide command line access to a FileSystem. */
 @InterfaceAudience.Private
@@ -54,13 +51,12 @@ public class FsShell extends Configured implements Tool {
   private FileSystem fs;
   private Trash trash;
   protected CommandFactory commandFactory;
-  private Sampler traceSampler;
 
   private final String usagePrefix =
     "Usage: hadoop fs [generic options]";
 
-  private SpanReceiverHost spanReceiverHost;
-  static final String SEHLL_HTRACE_PREFIX = "dfs.shell.htrace.";
+  private Tracer tracer;
+  static final String SHELL_HTRACE_PREFIX = "fs.shell.htrace.";
 
   /**
    * Default ctor with no configuration.  Be sure to invoke
@@ -102,8 +98,9 @@ public class FsShell extends Configured implements Tool {
       commandFactory.addObject(new Usage(), "-usage");
       registerCommands(commandFactory);
     }
-    this.spanReceiverHost =
-        SpanReceiverHost.get(getConf(), SEHLL_HTRACE_PREFIX);
+    this.tracer = new Tracer.Builder("FsShell").
+        conf(TraceUtils.wrapHadoopConf(SHELL_HTRACE_PREFIX, getConf())).
+        build();
   }
 
   protected void registerCommands(CommandFactory factory) {
@@ -285,8 +282,6 @@ public class FsShell extends Configured implements Tool {
   public int run(String argv[]) throws Exception {
     // initialize FsShell
     init();
-    traceSampler = new SamplerBuilder(TraceUtils.
-        wrapHadoopConf(SEHLL_HTRACE_PREFIX, getConf())).build();
     int exitCode = -1;
     if (argv.length < 1) {
       printUsage(System.err);
@@ -298,7 +293,7 @@ public class FsShell extends Configured implements Tool {
         if (instance == null) {
           throw new UnknownCommandException();
         }
-        TraceScope scope = Trace.startSpan(instance.getCommandName(), traceSampler);
+        TraceScope scope = tracer.newScope(instance.getCommandName());
         if (scope.getSpan() != null) {
           String args = StringUtils.join(" ", argv);
           if (args.length() > 2048) {
@@ -324,6 +319,7 @@ public class FsShell extends Configured implements Tool {
         e.printStackTrace(System.err);
       }
     }
+    tracer.close();
     return exitCode;
   }
   
@@ -350,9 +346,6 @@ public class FsShell extends Configured implements Tool {
       fs.close();
       fs = null;
     }
-    if (this.spanReceiverHost != null) {
-      this.spanReceiverHost.closeReceivers();
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java
new file mode 100644
index 0000000..e422336
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.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.fs;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tracing.TraceUtils;
+import org.apache.htrace.core.Tracer;
+
+/**
+ * Holds the HTrace Tracer used for FileSystem operations.
+ *
+ * Ideally, this would be owned by the DFSClient, rather than global.  However,
+ * the FileContext API may create a new DFSClient for each operation in some
+ * cases.  Because of this, we cannot store this Tracer inside DFSClient.  See
+ * HADOOP-6356 for details.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class FsTracer {
+  private static Tracer instance;
+
+  public static synchronized Tracer get(Configuration conf) {
+    if (instance == null) {
+      instance = new Tracer.Builder("FSClient").
+          conf(TraceUtils.wrapHadoopConf(CommonConfigurationKeys.
+              FS_CLIENT_HTRACE_PREFIX, conf)).
+          build();
+    }
+    return instance;
+  }
+
+  @VisibleForTesting
+  public static synchronized void clear() {
+    if (instance == null) {
+      return;
+    }
+    try {
+      instance.close();
+    } finally {
+      instance = null;
+    }
+  }
+
+  private FsTracer() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
index 48639b4..7a01575 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
@@ -28,9 +28,8 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -41,12 +40,14 @@ class Globber {
   private final FileContext fc;
   private final Path pathPattern;
   private final PathFilter filter;
+  private final Tracer tracer;
   
   public Globber(FileSystem fs, Path pathPattern, PathFilter filter) {
     this.fs = fs;
     this.fc = null;
     this.pathPattern = pathPattern;
     this.filter = filter;
+    this.tracer = fs.getTracer();
   }
 
   public Globber(FileContext fc, Path pathPattern, PathFilter filter) {
@@ -54,6 +55,7 @@ class Globber {
     this.fc = fc;
     this.pathPattern = pathPattern;
     this.filter = filter;
+    this.tracer = fc.getTracer();
   }
 
   private FileStatus getFileStatus(Path path) throws IOException {
@@ -140,11 +142,8 @@ class Globber {
   }
 
   public FileStatus[] glob() throws IOException {
-    TraceScope scope = Trace.startSpan("Globber#glob");
-    Span span = scope.getSpan();
-    if (span != null) {
-      span.addKVAnnotation("pattern", pathPattern.toUri().getPath());
-    }
+    TraceScope scope = tracer.newScope("Globber#glob");
+    scope.addKVAnnotation("pattern", pathPattern.toUri().getPath());
     try {
       return doGlob();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 9087e5c..bfa85aa 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -92,7 +92,8 @@ import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.Trace;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.Tracer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -722,8 +723,9 @@ public class Client {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Connecting to "+server);
         }
-        if (Trace.isTracing()) {
-          Trace.addTimelineAnnotation("IPC client connecting to " + server);
+        Span span = Tracer.getCurrentSpan();
+        if (span != null) {
+          span.addTimelineAnnotation("IPC client connecting to " + server);
         }
         short numRetries = 0;
         Random rand = null;
@@ -796,8 +798,9 @@ public class Client {
           // update last activity time
           touch();
 
-          if (Trace.isTracing()) {
-            Trace.addTimelineAnnotation("IPC client connected to " + server);
+          span = Tracer.getCurrentSpan();
+          if (span != null) {
+            span.addTimelineAnnotation("IPC client connected to " + server);
           }
 
           // start the receiver thread after the socket connection has been set

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 532246d..692d2b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -49,8 +49,8 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
@@ -206,12 +206,13 @@ public class ProtobufRpcEngine implements RpcEngine {
             + method.getName() + "]");
       }
 
-      TraceScope traceScope = null;
       // if Tracing is on then start a new span for this rpc.
       // guard it in the if statement to make sure there isn't
       // any extra string manipulation.
-      if (Trace.isTracing()) {
-        traceScope = Trace.startSpan(RpcClientUtil.methodToTraceString(method));
+      Tracer tracer = Tracer.curThreadTracer();
+      TraceScope traceScope = null;
+      if (tracer != null) {
+        traceScope = tracer.newScope(RpcClientUtil.methodToTraceString(method));
       }
 
       RequestHeaderProto rpcRequestHeader = constructRpcRequestHeader(method);
@@ -236,9 +237,9 @@ public class ProtobufRpcEngine implements RpcEngine {
               remoteId + ": " + method.getName() +
                 " {" + e + "}");
         }
-        if (Trace.isTracing()) {
-          traceScope.getSpan().addTimelineAnnotation(
-              "Call got exception: " + e.toString());
+        if (traceScope != null) {
+          traceScope.addTimelineAnnotation("Call got exception: " +
+              e.toString());
         }
         throw new ServiceException(e);
       } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 0f9ae43..b8026c6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -117,10 +117,9 @@ import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceInfo;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
@@ -141,6 +140,7 @@ public abstract class Server {
   private List<AuthMethod> enabledAuthMethods;
   private RpcSaslProto negotiateResponse;
   private ExceptionsHandler exceptionsHandler = new ExceptionsHandler();
+  private Tracer tracer;
   
   public void addTerseExceptions(Class<?>... exceptionClass) {
     exceptionsHandler.addTerseExceptions(exceptionClass);
@@ -581,7 +581,7 @@ public abstract class Server {
     private ByteBuffer rpcResponse;       // the response for this call
     private final RPC.RpcKind rpcKind;
     private final byte[] clientId;
-    private final Span traceSpan; // the tracing span on the server side
+    private final TraceScope traceScope; // the HTrace scope on the server side
 
     public Call(int id, int retryCount, Writable param, 
         Connection connection) {
@@ -595,7 +595,7 @@ public abstract class Server {
     }
 
     public Call(int id, int retryCount, Writable param, Connection connection,
-        RPC.RpcKind kind, byte[] clientId, Span span) {
+        RPC.RpcKind kind, byte[] clientId, TraceScope traceScope) {
       this.callId = id;
       this.retryCount = retryCount;
       this.rpcRequest = param;
@@ -604,7 +604,7 @@ public abstract class Server {
       this.rpcResponse = null;
       this.rpcKind = kind;
       this.clientId = clientId;
-      this.traceSpan = span;
+      this.traceScope = traceScope;
     }
     
     @Override
@@ -2014,19 +2014,24 @@ public abstract class Server {
             RpcErrorCodeProto.FATAL_DESERIALIZING_REQUEST, err);
       }
         
-      Span traceSpan = null;
+      TraceScope traceScope = null;
       if (header.hasTraceInfo()) {
-        // If the incoming RPC included tracing info, always continue the trace
-        TraceInfo parentSpan = new TraceInfo(header.getTraceInfo().getTraceId(),
-                                             header.getTraceInfo().getParentId());
-        traceSpan = Trace.startSpan(
-            RpcClientUtil.toTraceName(rpcRequest.toString()),
-            parentSpan).detach();
+        if (tracer != null) {
+          // If the incoming RPC included tracing info, always continue the
+          // trace
+          SpanId parentSpanId = new SpanId(
+              header.getTraceInfo().getTraceId(),
+              header.getTraceInfo().getParentId());
+          traceScope = tracer.newScope(
+              RpcClientUtil.toTraceName(rpcRequest.toString()),
+              parentSpanId);
+          traceScope.detach();
+        }
       }
 
       Call call = new Call(header.getCallId(), header.getRetryCount(),
           rpcRequest, this, ProtoUtil.convert(header.getRpcKind()),
-          header.getClientId().toByteArray(), traceSpan);
+          header.getClientId().toByteArray(), traceScope);
 
       if (callQueue.isClientBackoffEnabled()) {
         // if RPC queue is full, we will ask the RPC client to back off by
@@ -2209,8 +2214,9 @@ public abstract class Server {
           Writable value = null;
 
           CurCall.set(call);
-          if (call.traceSpan != null) {
-            traceScope = Trace.continueSpan(call.traceSpan);
+          if (call.traceScope != null) {
+            call.traceScope.reattach();
+            traceScope = call.traceScope;
             traceScope.getSpan().addTimelineAnnotation("called");
           }
 
@@ -2287,21 +2293,18 @@ public abstract class Server {
         } catch (InterruptedException e) {
           if (running) {                          // unexpected -- log it
             LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
-            if (Trace.isTracing()) {
+            if (traceScope != null) {
               traceScope.getSpan().addTimelineAnnotation("unexpectedly interrupted: " +
                   StringUtils.stringifyException(e));
             }
           }
         } catch (Exception e) {
           LOG.info(Thread.currentThread().getName() + " caught an exception", e);
-          if (Trace.isTracing()) {
+          if (traceScope != null) {
             traceScope.getSpan().addTimelineAnnotation("Exception: " +
                 StringUtils.stringifyException(e));
           }
         } finally {
-          if (traceScope != null) {
-            traceScope.close();
-          }
           IOUtils.cleanup(LOG, traceScope);
         }
       }
@@ -2615,6 +2618,10 @@ public abstract class Server {
   /** Sets the socket buffer size used for responding to RPCs */
   public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
 
+  public void setTracer(Tracer t) {
+    this.tracer = t;
+  }
+
   /** Starts the service.  Must be called before any calls will be handled. */
   public synchronized void start() {
     responder.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index 85490f3..46f33ba 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -42,8 +42,8 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.*;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 /** An RpcEngine implementation for Writable data. */
 @InterfaceStability.Evolving
@@ -233,9 +233,14 @@ public class WritableRpcEngine implements RpcEngine {
       if (LOG.isDebugEnabled()) {
         startTime = Time.now();
       }
+
+      // if Tracing is on then start a new span for this rpc.
+      // guard it in the if statement to make sure there isn't
+      // any extra string manipulation.
+      Tracer tracer = Tracer.curThreadTracer();
       TraceScope traceScope = null;
-      if (Trace.isTracing()) {
-        traceScope = Trace.startSpan(RpcClientUtil.methodToTraceString(method));
+      if (tracer != null) {
+        traceScope = tracer.newScope(RpcClientUtil.methodToTraceString(method));
       }
       ObjectWritable value;
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
deleted file mode 100644
index 9a99e05..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
+++ /dev/null
@@ -1,208 +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.tracing;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-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.tracing.SpanReceiverInfo.ConfigurationPair;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.htrace.SpanReceiver;
-import org.apache.htrace.SpanReceiverBuilder;
-import org.apache.htrace.Trace;
-import org.apache.htrace.impl.LocalFileSpanReceiver;
-
-/**
- * This class provides functions for reading the names of SpanReceivers from
- * the Hadoop configuration, adding those SpanReceivers to the Tracer,
- * and closing those SpanReceivers when appropriate.
- * This class does nothing If no SpanReceiver is configured.
- */
-@InterfaceAudience.Private
-public class SpanReceiverHost implements TraceAdminProtocol {
-  public static final String SPAN_RECEIVERS_CONF_SUFFIX =
-      "spanreceiver.classes";
-  private static final Log LOG = LogFactory.getLog(SpanReceiverHost.class);
-  private static final HashMap<String, SpanReceiverHost> hosts =
-      new HashMap<String, SpanReceiverHost>(1);
-  private final TreeMap<Long, SpanReceiver> receivers =
-      new TreeMap<Long, SpanReceiver>();
-  private final String confPrefix;
-  private Configuration config;
-  private boolean closed = false;
-  private long highestId = 1;
-
-  private final static String LOCAL_FILE_SPAN_RECEIVER_PATH_SUFFIX =
-      "local-file-span-receiver.path";
-
-  public static SpanReceiverHost get(Configuration conf, String confPrefix) {
-    synchronized (SpanReceiverHost.class) {
-      SpanReceiverHost host = hosts.get(confPrefix);
-      if (host != null) {
-        return host;
-      }
-      final SpanReceiverHost newHost = new SpanReceiverHost(confPrefix);
-      newHost.loadSpanReceivers(conf);
-      ShutdownHookManager.get().addShutdownHook(new Runnable() {
-          public void run() {
-            newHost.closeReceivers();
-          }
-        }, 0);
-      hosts.put(confPrefix, newHost);
-      return newHost;
-    }
-  }
-
-  private static List<ConfigurationPair> EMPTY = Collections.emptyList();
-
-  private SpanReceiverHost(String confPrefix) {
-    this.confPrefix = confPrefix;
-  }
-
-  /**
-   * Reads the names of classes specified in the
-   * "hadoop.htrace.spanreceiver.classes" property and instantiates and registers
-   * them with the Tracer as SpanReceiver's.
-   *
-   * The nullary constructor is called during construction, but if the classes
-   * specified implement the Configurable interface, setConfiguration() will be
-   * called on them. This allows SpanReceivers to use values from the Hadoop
-   * configuration.
-   */
-  public synchronized void loadSpanReceivers(Configuration conf) {
-    config = new Configuration(conf);
-    String receiverKey = confPrefix + SPAN_RECEIVERS_CONF_SUFFIX;
-    String[] receiverNames = config.getTrimmedStrings(receiverKey);
-    if (receiverNames == null || receiverNames.length == 0) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("No span receiver names found in " + receiverKey + ".");
-      }
-      return;
-    }
-    // It's convenient to have each daemon log to a random trace file when
-    // testing.
-    String pathKey = confPrefix + LOCAL_FILE_SPAN_RECEIVER_PATH_SUFFIX;
-    if (config.get(pathKey) == null) {
-      String uniqueFile = LocalFileSpanReceiver.getUniqueLocalTraceFileName();
-      config.set(pathKey, uniqueFile);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Set " + pathKey + " to " + uniqueFile);
-      }
-    }
-    for (String className : receiverNames) {
-      try {
-        SpanReceiver rcvr = loadInstance(className, EMPTY);
-        Trace.addReceiver(rcvr);
-        receivers.put(highestId++, rcvr);
-        LOG.info("Loaded SpanReceiver " + className + " successfully.");
-      } catch (IOException e) {
-        LOG.error("Failed to load SpanReceiver", e);
-      }
-    }
-  }
-
-  private synchronized SpanReceiver loadInstance(String className,
-      List<ConfigurationPair> extraConfig) throws IOException {
-    SpanReceiverBuilder builder =
-        new SpanReceiverBuilder(TraceUtils.
-            wrapHadoopConf(confPrefix, config, extraConfig));
-    SpanReceiver rcvr = builder.spanReceiverClass(className.trim()).build();
-    if (rcvr == null) {
-      throw new IOException("Failed to load SpanReceiver " + className);
-    }
-    return rcvr;
-  }
-
-  /**
-   * Calls close() on all SpanReceivers created by this SpanReceiverHost.
-   */
-  public synchronized void closeReceivers() {
-    if (closed) return;
-    closed = true;
-    for (SpanReceiver rcvr : receivers.values()) {
-      try {
-        rcvr.close();
-      } catch (IOException e) {
-        LOG.warn("Unable to close SpanReceiver correctly: " + e.getMessage(), e);
-      }
-    }
-    receivers.clear();
-  }
-
-  public synchronized SpanReceiverInfo[] listSpanReceivers()
-      throws IOException {
-    SpanReceiverInfo[] info = new SpanReceiverInfo[receivers.size()];
-    int i = 0;
-
-    for(Map.Entry<Long, SpanReceiver> entry : receivers.entrySet()) {
-      info[i] = new SpanReceiverInfo(entry.getKey(),
-          entry.getValue().getClass().getName());
-      i++;
-    }
-    return info;
-  }
-
-  public synchronized long addSpanReceiver(SpanReceiverInfo info)
-      throws IOException {
-    StringBuilder configStringBuilder = new StringBuilder();
-    String prefix = "";
-    for (ConfigurationPair pair : info.configPairs) {
-      configStringBuilder.append(prefix).append(pair.getKey()).
-          append(" = ").append(pair.getValue());
-      prefix = ", ";
-    }
-    SpanReceiver rcvr = null;
-    try {
-      rcvr = loadInstance(info.getClassName(), info.configPairs);
-    } catch (IOException e) {
-      LOG.info("Failed to add SpanReceiver " + info.getClassName() +
-          " with configuration " + configStringBuilder.toString(), e);
-      throw e;
-    } catch (RuntimeException e) {
-      LOG.info("Failed to add SpanReceiver " + info.getClassName() +
-          " with configuration " + configStringBuilder.toString(), e);
-      throw e;
-    }
-    Trace.addReceiver(rcvr);
-    long newId = highestId++;
-    receivers.put(newId, rcvr);
-    LOG.info("Successfully added SpanReceiver " + info.getClassName() +
-        " with configuration " + configStringBuilder.toString());
-    return newId;
-  }
-
-  public synchronized void removeSpanReceiver(long spanReceiverId)
-      throws IOException {
-    SpanReceiver rcvr = receivers.remove(spanReceiverId);
-    if (rcvr == null) {
-      throw new IOException("There is no span receiver with id " + spanReceiverId);
-    }
-    Trace.removeReceiver(rcvr);
-    rcvr.close();
-    LOG.info("Successfully removed SpanReceiver " + spanReceiverId +
-        " with class " + rcvr.getClass().getName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
index 52b5d47..09acb35 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
-import org.apache.htrace.HTraceConfiguration;
+import org.apache.htrace.core.HTraceConfiguration;
 
 /**
  * This class provides utility functions for tracing.
@@ -32,6 +32,7 @@ import org.apache.htrace.HTraceConfiguration;
 @InterfaceAudience.Private
 public class TraceUtils {
   private static List<ConfigurationPair> EMPTY = Collections.emptyList();
+  static final String DEFAULT_HADOOP_PREFIX = "hadoop.htrace.";
 
   public static HTraceConfiguration wrapHadoopConf(final String prefix,
         final Configuration conf) {
@@ -47,16 +48,27 @@ public class TraceUtils {
     return new HTraceConfiguration() {
       @Override
       public String get(String key) {
-        return get(key, "");
+        String ret = getInternal(prefix + key);
+        if (ret != null) {
+          return ret;
+        }
+        return getInternal(DEFAULT_HADOOP_PREFIX  + key);
       }
 
       @Override
       public String get(String key, String defaultValue) {
-        String prefixedKey = prefix + key;
-        if (extraMap.containsKey(prefixedKey)) {
-          return extraMap.get(prefixedKey);
+        String ret = get(key);
+        if (ret != null) {
+          return ret;
+        }
+        return defaultValue;
+      }
+
+      private String getInternal(String key) {
+        if (extraMap.containsKey(key)) {
+          return extraMap.get(key);
         }
-        return conf.get(prefixedKey, defaultValue);
+        return conf.get(key);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TracerConfigurationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TracerConfigurationManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TracerConfigurationManager.java
new file mode 100644
index 0000000..75601ad
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TracerConfigurationManager.java
@@ -0,0 +1,100 @@
+/**
+ * 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.tracing;
+
+import java.io.IOException;
+
+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.tracing.SpanReceiverInfo.ConfigurationPair;
+import org.apache.htrace.core.SpanReceiver;
+import org.apache.htrace.core.TracerPool;
+
+/**
+ * This class provides functions for managing the tracer configuration at
+ * runtime via an RPC protocol.
+ */
+@InterfaceAudience.Private
+public class TracerConfigurationManager implements TraceAdminProtocol {
+  private static final Log LOG =
+      LogFactory.getLog(TracerConfigurationManager.class);
+
+  private final String confPrefix;
+  private final Configuration conf;
+
+  public TracerConfigurationManager(String confPrefix, Configuration conf) {
+    this.confPrefix = confPrefix;
+    this.conf = conf;
+  }
+
+  public synchronized SpanReceiverInfo[] listSpanReceivers()
+      throws IOException {
+    TracerPool pool = TracerPool.getGlobalTracerPool();
+    SpanReceiver[] receivers = pool.getReceivers();
+    SpanReceiverInfo[] info = new SpanReceiverInfo[receivers.length];
+    for (int i = 0; i < receivers.length; i++) {
+      SpanReceiver receiver = receivers[i];
+      info[i] = new SpanReceiverInfo(receiver.getId(),
+          receiver.getClass().getName());
+    }
+    return info;
+  }
+
+  public synchronized long addSpanReceiver(SpanReceiverInfo info)
+      throws IOException {
+    StringBuilder configStringBuilder = new StringBuilder();
+    String prefix = "";
+    for (ConfigurationPair pair : info.configPairs) {
+      configStringBuilder.append(prefix).append(pair.getKey()).
+          append(" = ").append(pair.getValue());
+      prefix = ", ";
+    }
+    SpanReceiver rcvr = null;
+    try {
+      rcvr = new SpanReceiver.Builder(TraceUtils.wrapHadoopConf(
+              confPrefix, conf, info.configPairs)).
+          className(info.getClassName().trim()).
+          build();
+    } catch (RuntimeException e) {
+      LOG.info("Failed to add SpanReceiver " + info.getClassName() +
+          " with configuration " + configStringBuilder.toString(), e);
+      throw e;
+    }
+    TracerPool.getGlobalTracerPool().addReceiver(rcvr);
+    LOG.info("Successfully added SpanReceiver " + info.getClassName() +
+        " with configuration " + configStringBuilder.toString());
+    return rcvr.getId();
+  }
+
+  public synchronized void removeSpanReceiver(long spanReceiverId)
+      throws IOException {
+    SpanReceiver[] receivers =
+        TracerPool.getGlobalTracerPool().getReceivers();
+    for (SpanReceiver receiver : receivers) {
+      if (receiver.getId() == spanReceiverId) {
+        TracerPool.getGlobalTracerPool().removeAndCloseReceiver(receiver);
+        LOG.info("Successfully removed SpanReceiver " + spanReceiverId +
+            " with class " + receiver.getClass().getName());
+        return;
+      }
+    }
+    throw new IOException("There is no span receiver with id " + spanReceiverId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
index 4b3b7ef..3a6bf90 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformation
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.Tracer;
 
 import com.google.protobuf.ByteString;
 
@@ -169,11 +169,12 @@ public abstract class ProtoUtil {
         .setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid));
 
     // Add tracing info if we are currently tracing.
-    if (Trace.isTracing()) {
-      Span s = Trace.currentSpan();
+    Span span = Tracer.getCurrentSpan();
+    if (span != null) {
       result.setTraceInfo(RPCTraceInfoProto.newBuilder()
-          .setParentId(s.getSpanId())
-          .setTraceId(s.getTraceId()).build());
+          .setTraceId(span.getSpanId().getHigh())
+          .setParentId(span.getSpanId().getLow())
+            .build());
     }
 
     return result.build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
index c879150..7546c62 100644
--- a/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
+++ b/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
@@ -61,8 +61,9 @@ enum RpcKindProto {
  * what span caused the new span we will create when this message is received.
  */
 message RPCTraceInfoProto {
-  optional int64 traceId = 1;
-  optional int64 parentId = 2;
+    optional int64 traceId = 1; // parentIdHigh
+    optional int64 parentId = 2; // parentIdLow
+
 }
 
 message RpcRequestHeaderProto { // the header for the RpcRequest

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index a57e81b..9af86e0 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1998,4 +1998,19 @@ for ldap providers in the same way as above does.
       the limit is 0 or the -safely is not specified in -rm command.
     </description>
   </property>
+
+  <property>
+    <name>fs.client.htrace.sampler.classes</name>
+    <value></value>
+    <description>The class names of the HTrace Samplers to use for Hadoop
+      filesystem clients.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.htrace.span.receiver.classes</name>
+    <value></value>
+    <description>The class names of the Span Receivers to use for Hadoop.
+    </description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md b/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
index bf5685c..7897855 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Tracing.md
@@ -49,37 +49,14 @@ interface bundled with HTrace or implementing it by yourself.
 * HTracedRESTReceiver
 * ZipkinSpanReceiver
 
-In order to set up SpanReceivers for HDFS servers,
-configure what SpanReceivers you'd like to use
-by putting a comma separated list of the fully-qualified class name of classes implementing SpanReceiver
-in `hdfs-site.xml` property: `dfs.htrace.spanreceiver.classes`.
+See core-default.xml for a description of HTrace configuration keys.  In some
+cases, you will also need to add the jar containing the SpanReceiver that you
+are using to the classpath of Hadoop on each node. (In the example above,
+LocalFileSpanReceiver is included in the htrace-core4 jar which is bundled
+with Hadoop.)
 
-```xml
-      <property>
-        <name>dfs.htrace.spanreceiver.classes</name>
-        <value>org.apache.htrace.impl.LocalFileSpanReceiver</value>
-      </property>
-      <property>
-        <name>dfs.htrace.local-file-span-receiver.path</name>
-        <value>/var/log/hadoop/htrace.out</value>
-      </property>
-```
-
-You can omit package name prefix if you use span receiver bundled with HTrace.
-
-```xml
-      <property>
-        <name>dfs.htrace.spanreceiver.classes</name>
-        <value>LocalFileSpanReceiver</value>
-      </property>
 ```
-
-You also need to add the jar bundling SpanReceiver to the classpath of Hadoop
-on each node. (LocalFileSpanReceiver in the example above is included in the
-jar of htrace-core which is bundled with Hadoop.)
-
-```
-    $ cp htrace-htraced/target/htrace-htraced-3.2.0-incubating.jar $HADOOP_HOME/share/hadoop/common/lib/
+    $ cp htrace-htraced/target/htrace-htraced-4.0.1-incubating.jar $HADOOP_HOME/share/hadoop/common/lib/
 ```
 
 ### Dynamic update of tracing configuration
@@ -92,11 +69,11 @@ You need to run the command against all servers if you want to update the config
 
       $ hadoop trace -list -host 192.168.56.2:9000
       ID  CLASS
-      1   org.apache.htrace.impl.LocalFileSpanReceiver
+      1   org.apache.htrace.core.LocalFileSpanReceiver
 
       $ hadoop trace -list -host 192.168.56.2:50020
       ID  CLASS
-      1   org.apache.htrace.impl.LocalFileSpanReceiver
+      1   org.apache.htrace.core.LocalFileSpanReceiver
 
 `hadoop trace -remove` removes span receiver from server.
 `-remove` options takes id of span receiver as argument.
@@ -113,7 +90,7 @@ You can specify the configuration associated with span receiver by `-Ckey=value`
 
       $ hadoop trace -list -host 192.168.56.2:9000
       ID  CLASS
-      2   org.apache.htrace.impl.LocalFileSpanReceiver
+      2   org.apache.htrace.core.LocalFileSpanReceiver
 
 ### Starting tracing spans by HTrace API
 
@@ -121,26 +98,21 @@ In order to trace, you will need to wrap the traced logic with **tracing span**
 When there is running tracing spans,
 the tracing information is propagated to servers along with RPC requests.
 
-In addition, you need to initialize `SpanReceiverHost` once per process.
-
 ```java
     import org.apache.hadoop.hdfs.HdfsConfiguration;
-    import org.apache.hadoop.tracing.SpanReceiverHost;
-    import org.apache.htrace.Sampler;
-    import org.apache.htrace.Trace;
-    import org.apache.htrace.TraceScope;
+    import org.apache.htrace.core.Tracer;
+    import org.apache.htrace.core.TraceScope;
 
     ...
 
-        SpanReceiverHost.getInstance(new HdfsConfiguration());
 
     ...
 
-        TraceScope ts = Trace.startSpan("Gets", Sampler.ALWAYS);
+        TraceScope ts = tracer.newScope("Gets");
         try {
           ... // traced logic
         } finally {
-          if (ts != null) ts.close();
+          ts.close();
         }
 ```
 
@@ -154,11 +126,10 @@ which start tracing span before invoking HDFS shell command.
     import org.apache.hadoop.fs.FsShell;
     import org.apache.hadoop.hdfs.DFSConfigKeys;
     import org.apache.hadoop.hdfs.HdfsConfiguration;
-    import org.apache.hadoop.tracing.SpanReceiverHost;
+    import org.apache.hadoop.tracing.TraceUtils;
     import org.apache.hadoop.util.ToolRunner;
-    import org.apache.htrace.Sampler;
-    import org.apache.htrace.Trace;
-    import org.apache.htrace.TraceScope;
+    import org.apache.htrace.core.Trace;
+    import org.apache.htrace.core.TraceScope;
 
     public class TracingFsShell {
       public static void main(String argv[]) throws Exception {
@@ -166,13 +137,19 @@ which start tracing span before invoking HDFS shell command.
         FsShell shell = new FsShell();
         conf.setQuietMode(false);
         shell.setConf(conf);
-        SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX);
+        Tracer tracer = new Tracer.Builder().
+            name("TracingFsShell).
+            conf(TraceUtils.wrapHadoopConf("tracing.fs.shell.htrace.", conf)).
+            build();
         int res = 0;
-        try (TraceScope ts = Trace.startSpan("FsShell", Sampler.ALWAYS)) {
+        TraceScope scope = tracer.newScope("FsShell");
+        try {
           res = ToolRunner.run(shell, argv);
         } finally {
+          scope.close();
           shell.close();
         }
+        tracer.close();
         System.exit(res);
       }
     }
@@ -189,16 +166,15 @@ The DFSClient can enable tracing internally. This allows you to use HTrace with
 your client without modifying the client source code.
 
 Configure the span receivers and samplers in `hdfs-site.xml`
-by properties `dfs.client.htrace.sampler` and `dfs.client.htrace.sampler`.
-The value of `dfs.client.htrace.sampler` can be NeverSampler, AlwaysSampler or ProbabilitySampler.
+by properties `fs.client.htrace.sampler.classes` and
+`fs.client.htrace.spanreceiver.classes`.  The value of
+`fs.client.htrace.sampler.classes` can be NeverSampler, AlwaysSampler or
+ProbabilitySampler.
 
 * NeverSampler: HTrace is OFF for all requests to namenodes and datanodes;
 * AlwaysSampler: HTrace is ON for all requests to namenodes and datanodes;
 * ProbabilitySampler: HTrace is ON for some percentage% of  requests to namenodes and datanodes
 
-You do not need to enable this if your client program has been modified
-to use HTrace.
-
 ```xml
       <property>
         <name>dfs.client.htrace.spanreceiver.classes</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShell.java
index cc93d68..376f8a6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShell.java
@@ -24,10 +24,9 @@ import junit.framework.AssertionFailedError;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.tracing.SetSpanReceiver;
-import org.apache.hadoop.tracing.SpanReceiverHost;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.SamplerBuilder;
-import org.apache.htrace.impl.AlwaysSampler;
+import org.apache.htrace.core.AlwaysSampler;
+import org.apache.htrace.core.Tracer;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -53,10 +52,10 @@ public class TestFsShell {
   @Test
   public void testTracing() throws Throwable {
     Configuration conf = new Configuration();
-    String prefix = FsShell.SEHLL_HTRACE_PREFIX;
-    conf.set(prefix + SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
+    String prefix = "fs.shell.htrace.";
+    conf.set(prefix + Tracer.SPAN_RECEIVER_CLASSES_KEY,
         SetSpanReceiver.class.getName());
-    conf.set(prefix + SamplerBuilder.SAMPLER_CONF_KEY,
+    conf.set(prefix + Tracer.SAMPLER_CLASSES_KEY,
         AlwaysSampler.class.getName());
     conf.setQuietMode(false);
     FsShell shell = new FsShell(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
index 97ca7c4..2bc68ce 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
@@ -19,9 +19,10 @@ package org.apache.hadoop.tracing;
 
 import com.google.common.base.Supplier;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.htrace.Span;
-import org.apache.htrace.SpanReceiver;
-import org.apache.htrace.HTraceConfiguration;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.core.SpanReceiver;
+import org.apache.htrace.core.HTraceConfiguration;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -39,7 +40,7 @@ import org.junit.Assert;
  * push all the metrics to a static place, and would make testing
  * SpanReceiverHost harder.
  */
-public class SetSpanReceiver implements SpanReceiver {
+public class SetSpanReceiver extends SpanReceiver {
 
   public SetSpanReceiver(HTraceConfiguration conf) {
   }
@@ -68,8 +69,8 @@ public class SetSpanReceiver implements SpanReceiver {
   }
 
   public static class SetHolder {
-    public static ConcurrentHashMap<Long, Span> spans =
-        new ConcurrentHashMap<Long, Span>();
+    public static ConcurrentHashMap<SpanId, Span> spans =
+        new ConcurrentHashMap<SpanId, Span>();
 
     public static Map<String, List<Span>> getMap() {
       Map<String, List<Span>> map = new HashMap<String, List<Span>>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
index 400d003..75411da 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
@@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals;
 import java.util.LinkedList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
-import org.apache.htrace.HTraceConfiguration;
+import org.apache.htrace.core.HTraceConfiguration;
 import org.junit.Test;
 
 public class TestTraceUtils {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/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
index 69e9da2..273d8cf 100644
--- 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
@@ -66,6 +66,7 @@ import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import org.apache.htrace.core.Tracer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -179,6 +180,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private Configuration configuration;
 
   /**
+   * The HTrace tracer to use.
+   */
+  private Tracer tracer;
+
+  /**
    * 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.
    */
@@ -282,6 +288,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     return this;
   }
 
+  public BlockReaderFactory setTracer(Tracer tracer) {
+    this.tracer = tracer;
+    return this;
+  }
+
   @VisibleForTesting
   public static void setFailureInjectorForTesting(FailureInjector injector) {
     failureInjector = injector;
@@ -435,7 +446,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     try {
       return BlockReaderLocalLegacy.newBlockReader(conf,
           userGroupInformation, configuration, fileName, block, token,
-          datanode, startOffset, length, storageType);
+          datanode, startOffset, length, storageType, tracer);
     } catch (RemoteException remoteException) {
       ioe = remoteException.unwrapRemoteException(
                 InvalidToken.class, AccessControlException.class);
@@ -496,6 +507,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         setVerifyChecksum(verifyChecksum).
         setCachingStrategy(cachingStrategy).
         setStorageType(storageType).
+        setTracer(tracer).
         build();
   }
 
@@ -865,12 +877,12 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       return RemoteBlockReader.newBlockReader(fileName,
           block, token, startOffset, length, conf.getIoBufferSize(),
           verifyChecksum, clientName, peer, datanode,
-          clientContext.getPeerCache(), cachingStrategy);
+          clientContext.getPeerCache(), cachingStrategy, tracer);
     } else {
       return RemoteBlockReader2.newBlockReader(
           fileName, block, token, startOffset, length,
           verifyChecksum, clientName, peer, datanode,
-          clientContext.getPeerCache(), cachingStrategy);
+          clientContext.getPeerCache(), cachingStrategy, tracer);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index 2a0e21b..62e7af6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@ -34,9 +34,8 @@ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DirectBufferPool;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -75,6 +74,7 @@ class BlockReaderLocal implements BlockReader {
     private long dataPos;
     private ExtendedBlock block;
     private StorageType storageType;
+    private Tracer tracer;
 
     public Builder(ShortCircuitConf conf) {
       this.maxReadahead = Integer.MAX_VALUE;
@@ -120,6 +120,11 @@ class BlockReaderLocal implements BlockReader {
       return this;
     }
 
+    public Builder setTracer(Tracer tracer) {
+      this.tracer = tracer;
+      return this;
+    }
+
     public BlockReaderLocal build() {
       Preconditions.checkNotNull(replica);
       return new BlockReaderLocal(this);
@@ -228,6 +233,11 @@ class BlockReaderLocal implements BlockReader {
    */
   private StorageType storageType;
 
+  /**
+   * The Tracer to use.
+   */
+  private final Tracer tracer;
+
   private BlockReaderLocal(Builder builder) {
     this.replica = builder.replica;
     this.dataIn = replica.getDataStream().getChannel();
@@ -257,6 +267,7 @@ class BlockReaderLocal implements BlockReader {
     }
     this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum;
     this.storageType = builder.storageType;
+    this.tracer = builder.tracer;
   }
 
   private synchronized void createDataBufIfNeeded() {
@@ -324,8 +335,8 @@ class BlockReaderLocal implements BlockReader {
    */
   private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
       throws IOException {
-    TraceScope scope = Trace.startSpan("BlockReaderLocal#fillBuffer(" +
-        block.getBlockId() + ")", Sampler.NEVER);
+    TraceScope scope = tracer.newScope(
+        "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")");
     try {
       int total = 0;
       long startDataPos = dataPos;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/892ade68/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index eea3f06..5235287 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -50,10 +50,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DirectBufferPool;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -182,7 +180,8 @@ class BlockReaderLocalLegacy implements BlockReader {
   private long startOffset;
   private final String filename;
   private long blockId;
-  
+  private final Tracer tracer;
+
   /**
    * The only way this object can be instantiated.
    */
@@ -190,8 +189,8 @@ class BlockReaderLocalLegacy implements BlockReader {
       UserGroupInformation userGroupInformation,
       Configuration configuration, String file, ExtendedBlock blk,
       Token<BlockTokenIdentifier> token, DatanodeInfo node, 
-      long startOffset, long length, StorageType storageType)
-      throws IOException {
+      long startOffset, long length, StorageType storageType,
+      Tracer tracer) throws IOException {
     final ShortCircuitConf scConf = conf.getShortCircuitConf();
     LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
         .getIpcPort());
@@ -239,10 +238,10 @@ class BlockReaderLocalLegacy implements BlockReader {
             - (startOffset % checksum.getBytesPerChecksum());
         localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
             startOffset, length, pathinfo, checksum, true, dataIn,
-            firstChunkOffset, checksumIn);
+            firstChunkOffset, checksumIn, tracer);
       } else {
         localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
-            startOffset, length, pathinfo, dataIn);
+            startOffset, length, pathinfo, dataIn, tracer);
       }
     } catch (IOException e) {
       // remove from cache
@@ -321,18 +320,18 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
       ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
-      long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
-      throws IOException {
+      long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn,
+      Tracer tracer) throws IOException {
     this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
         DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
-        dataIn, startOffset, null);
+        dataIn, startOffset, null, tracer);
   }
 
   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
       ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
       long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
       boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
-      FileInputStream checksumIn) throws IOException {
+      FileInputStream checksumIn, Tracer tracer) throws IOException {
     this.filename = hdfsfile;
     this.checksum = checksum;
     this.verifyChecksum = verifyChecksum;
@@ -368,6 +367,7 @@ class BlockReaderLocalLegacy implements BlockReader {
         bufferPool.returnBuffer(checksumBuff);
       }
     }
+    this.tracer = tracer;
   }
 
   /**
@@ -375,8 +375,8 @@ class BlockReaderLocalLegacy implements BlockReader {
    */
   private int fillBuffer(FileInputStream stream, ByteBuffer buf)
       throws IOException {
-    TraceScope scope = Trace.startSpan("BlockReaderLocalLegacy#fillBuffer(" +
-        blockId + ")", Sampler.NEVER);
+    TraceScope scope = tracer.
+        newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")");
     try {
       int bytesRead = stream.getChannel().read(buf);
       if (bytesRead < 0) {


[42/58] [abbrv] hadoop git commit: HDFS-9141. Thread leak in Datanode#refreshVolumes. (Uma Maheswara Rao G via yliu)

Posted by zh...@apache.org.
HDFS-9141. Thread leak in Datanode#refreshVolumes. (Uma Maheswara Rao G via yliu)


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

Branch: refs/heads/HDFS-7285
Commit: 715dbddf77866bb47a4b95421091f64a3785444f
Parents: d6fa34e
Author: yliu <yl...@apache.org>
Authored: Tue Sep 29 22:05:34 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Tue Sep 29 22:05:34 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt              |  3 +++
 .../org/apache/hadoop/hdfs/server/datanode/DataNode.java | 11 ++++++-----
 2 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/715dbddf/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 182464b..2c90b23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1453,6 +1453,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9092. Nfs silently drops overlapping write requests and causes data
     copying to fail. (Yongjun Zhang)
 
+    HDFS-9141. Thread leak in Datanode#refreshVolumes. (Uma Maheswara Rao G
+    via yliu)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/715dbddf/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 2646089..2fe67fd 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
@@ -25,9 +25,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISS
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY;
@@ -604,7 +602,7 @@ public class DataNode extends ReconfigurableBase
   private synchronized void refreshVolumes(String newVolumes) throws IOException {
     Configuration conf = getConf();
     conf.set(DFS_DATANODE_DATA_DIR_KEY, newVolumes);
-
+    ExecutorService service = null;
     int numOldDataDirs = dataDirs.size();
     ChangedVolumes changedVolumes = parseChangedVolumes(newVolumes);
     StringBuilder errorMessageBuilder = new StringBuilder();
@@ -627,8 +625,8 @@ public class DataNode extends ReconfigurableBase
         for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
           nsInfos.add(bpos.getNamespaceInfo());
         }
-        ExecutorService service = Executors.newFixedThreadPool(
-            changedVolumes.newLocations.size());
+        service = Executors
+            .newFixedThreadPool(changedVolumes.newLocations.size());
         List<Future<IOException>> exceptions = Lists.newArrayList();
         for (final StorageLocation location : changedVolumes.newLocations) {
           exceptions.add(service.submit(new Callable<IOException>() {
@@ -678,6 +676,9 @@ public class DataNode extends ReconfigurableBase
         throw new IOException(errorMessageBuilder.toString());
       }
     } finally {
+      if (service != null) {
+        service.shutdown();
+      }
       conf.set(DFS_DATANODE_DATA_DIR_KEY,
           Joiner.on(",").join(effectiveVolumes));
       dataDirs = getStorageLocations(conf);


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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 b3940b5..4ebf437 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -977,6 +977,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/bf37d3d8/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/bf37d3d8/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/bf37d3d8/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;
-  }
-}


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

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf37d3d8/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..67cd524
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -0,0 +1,3144 @@
+/**
+ * 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.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.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.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.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.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.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 {
+    checkOpen();
+    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 {
+    checkOpen();
+    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 {
+    checkOpen();
+    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 {
+    checkOpen();
+    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();
+    }
+  }
+  
+  /**
+   * 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 {
+    checkOpen();
+    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 {
+    checkOpen();
+    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 {
+    checkOpen();
+    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 {
+    checkOpen();
+    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 + ".");
+    }
+    TraceScope scope = getPathTraceScope("truncate", src);
+    try {
+      return namenode.truncate(src, newLength, clientName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * 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 {
+    checkOpen();
+    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 {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Rename a snapshot.
+   * @param snapshotDir The directory path where the snapshot was taken
+   * @param snapshotOldName Old name of the snapshot
+   * @param snapshotNewName New name of the snapshot
+   * @throws IOException
+   * @see ClientProtocol#renameSnapshot(String, String, String)
+   */
+  public void renameSnapshot(String snapshotDir, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("renameSnapshot", traceSampler);
+    try {
+      namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Get all the current snapshottable directories.
+   * @return All the current snapshottable directories
+   * @throws IOException
+   * @see ClientProtocol#getSnapshottableDirListing()
+   */
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("getSnapshottableDirListing",
+        traceSampler);
+    try {
+      return namenode.getSnapshottableDirListing();
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * Allow snapshot on a directory.
+   * 
+   * @see ClientProtocol#allowSnapshot(String snapshotRoot)
+   */
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("allowSnapshot", traceSampler);
+    try {
+      namenode.allowSnapshot(snapshotRoot);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Disallow snapshot on a directory.
+   * 
+   * @see ClientProtocol#disallowSnapshot(String snapshotRoot)
+   */
+  public void disallowSnapshot(String snapshotRoot) throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("disallowSnapshot", traceSampler);
+    try {
+      namenode.disallowSnapshot(snapshotRoot);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
+    }
+  }
+  
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   * @see ClientProtocol#getSnapshotDiffReport(String, String, String)
+   */
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("getSnapshotDiffReport", traceSampler);
+    try {
+      return namenode.getSnapshotDiffReport(snapshotDir,
+          fromSnapshot, toSnapshot);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
+    }
+  }
+
+  public long addCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("addCacheDirective", traceSampler);
+    try {
+      return namenode.addCacheDirective(info, flags);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
+    }
+  }
+  
+  public void modifyCacheDirective(
+      CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException 

<TRUNCATED>

[54/58] [abbrv] hadoop git commit: HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client package. Contributed by Mingliang Liu.

Posted by zh...@apache.org.
HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client package. 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/39285e6a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/39285e6a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/39285e6a

Branch: refs/heads/HDFS-7285
Commit: 39285e6a1978ea5e53bdc1b0aef62421382124a8
Parents: 6ee0539
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 29 17:52:36 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 29 17:52:36 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 114 ++++++-------------
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  77 ++++---------
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  20 +---
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  36 ++----
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  85 +++++---------
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  36 +++---
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  47 +++-----
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   4 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  25 ++--
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  18 +--
 .../protocol/datatransfer/PacketReceiver.java   |   8 +-
 .../hdfs/protocol/datatransfer/Sender.java      |   6 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java |   8 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  10 +-
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  45 +++-----
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 101 ++++------------
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  33 ++----
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  14 +--
 .../hadoop/hdfs/util/ByteArrayManager.java      |  63 ++--------
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |   8 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |   4 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  28 ++---
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 24 files changed, 230 insertions(+), 573 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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
index 4f37090..f249692 100644
--- 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
@@ -349,17 +349,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       if (clientContext.getUseLegacyBlockReaderLocal()) {
         reader = getLegacyBlockReaderLocal();
         if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new legacy block reader local.");
-          }
+          LOG.trace("{}: returning new legacy block reader local.", this);
           return reader;
         }
       } else {
         reader = getBlockReaderLocal();
         if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new block reader local.");
-          }
+          LOG.trace("{}: returning new block reader local.", this);
           return reader;
         }
       }
@@ -367,10 +363,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     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());
-        }
+        LOG.trace("{}: returning new remote block reader using UNIX domain "
+            + "socket on {}", this, pathInfo.getPath());
         return reader;
       }
     }
@@ -405,10 +399,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             setVisibleLength(visibleLength).
             build();
         if (accessor == null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": No ReplicaAccessor created by " +
-                cls.getName());
-          }
+          LOG.trace("{}: No ReplicaAccessor created by {}",
+              this, cls.getName());
         } else {
           return new ExternalBlockReader(accessor, visibleLength, startOffset);
         }
@@ -427,14 +419,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * first introduced in HDFS-2246.
    */
   private BlockReader getLegacyBlockReaderLocal() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
-    }
+    LOG.trace("{}: trying to construct BlockReaderLocalLegacy", this);
     if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
-            "the address " + inetSocketAddress + " is not local");
-      }
+      LOG.trace("{}: can't construct BlockReaderLocalLegacy because the address"
+          + "{} is not local", this, inetSocketAddress);
       return null;
     }
     if (clientContext.getDisableLegacyBlockReaderLocal()) {
@@ -470,10 +458,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   }
 
   private BlockReader getBlockReaderLocal() throws InvalidToken {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct a BlockReaderLocal " +
-          "for short-circuit reads.");
-    }
+    LOG.trace("{}: trying to construct a BlockReaderLocal for short-circuit "
+        + " reads.", this);
     if (pathInfo == null) {
       pathInfo = clientContext.getDomainSocketFactory()
           .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
@@ -488,10 +474,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     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());
-      }
+      LOG.trace("{}: got InvalidToken exception while trying to construct "
+          + "BlockReaderLocal via {}", this, pathInfo.getPath());
       throw exc;
     }
     if (info.getReplica() == null) {
@@ -527,9 +511,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
       if (info != null) return info;
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
-    }
+    LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
     BlockReaderPeer curPeer;
     while (true) {
       curPeer = nextDomainPeer();
@@ -544,10 +526,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             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...");
-          }
+          LOG.trace("{}: allocShmSlot used up our previous socket {}.  "
+              + "Allocating a new one...", this, peer.getDomainSocket());
           curPeer = nextDomainPeer();
           if (curPeer == null) break;
           peer = (DomainPeer)curPeer.peer;
@@ -562,9 +542,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         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);
-          }
+          LOG.debug("{}: closing stale domain peer {}", this, peer, e);
           IOUtilsClient.cleanup(LOG, peer);
         } else {
           // Handle an I/O error we got when using a newly created socket.
@@ -617,7 +595,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
         if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
-          LOG.trace("Sending receipt verification byte for slot " + slot);
+          LOG.trace("Sending receipt verification byte for slot {}", slot);
           sock.getOutputStream().write(0);
         }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
@@ -650,9 +628,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       String msg = "access control error while " +
           "attempting to set up short-circuit access to " +
           fileName + resp.getMessage();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(this + ":" + msg);
-      }
+      LOG.debug("{}:{}", this, msg);
       return new ShortCircuitReplicaInfo(new InvalidToken(msg));
     default:
       LOG.warn(this + ": unknown response code " + resp.getStatus() +
@@ -684,10 +660,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
            " 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());
-    }
+    LOG.trace("{}: trying to create a remote block reader from the UNIX domain "
+        + "socket at {}", this, pathInfo.getPath());
 
     while (true) {
       BlockReaderPeer curPeer = nextDomainPeer();
@@ -701,19 +675,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       } 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);
-          }
+          LOG.trace("{}: got security exception while constructing a remote "
+                  + " block reader from the unix domain socket at {}",
+              this, 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);
-          }
+          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
@@ -747,10 +717,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    *             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");
-    }
+    LOG.trace("{}: trying to create a remote block reader from a TCP socket",
+        this);
     BlockReader blockReader = null;
     while (true) {
       BlockReaderPeer curPeer = null;
@@ -763,19 +731,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         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);
-          }
+          LOG.trace("{}: got security exception while constructing a remote "
+              + "block reader from {}", this, 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);
-          }
+          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);
@@ -808,9 +772,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, true);
       if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextDomainPeer: reusing existing peer " + peer);
-        }
+        LOG.trace("nextDomainPeer: reusing existing peer {}", peer);
         return new BlockReaderPeer(peer, true);
       }
     }
@@ -832,24 +794,18 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, false);
       if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextTcpPeer: reusing existing peer " + peer);
-        }
+        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);
-      }
+      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);
-      }
+      LOG.trace("nextTcpPeer: failed to create newConnectedPeer connected to"
+          + "{}", datanode);
       throw e;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index 62e7af6..10dc35c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@ -412,17 +412,10 @@ class BlockReaderLocal implements BlockReader {
   public synchronized int read(ByteBuffer buf) throws IOException {
     boolean canSkipChecksum = createNoChecksumContext();
     try {
-      String traceString = null;
-      if (LOG.isTraceEnabled()) {
-        traceString = new StringBuilder().
-            append("read(").
-            append("buf.remaining=").append(buf.remaining()).
-            append(", block=").append(block).
-            append(", filename=").append(filename).
-            append(", canSkipChecksum=").append(canSkipChecksum).
-            append(")").toString();
-        LOG.info(traceString + ": starting");
-      }
+      String traceFormatStr = "read(buf.remaining={}, block={}, filename={}, "
+          + "canSkipChecksum={})";
+      LOG.trace(traceFormatStr + ": starting",
+          buf.remaining(), block, filename, canSkipChecksum);
       int nRead;
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
@@ -431,14 +424,12 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(buf, canSkipChecksum);
         }
       } catch (IOException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.info(traceString + ": I/O error", e);
-        }
+        LOG.trace(traceFormatStr + ": I/O error",
+            buf.remaining(), block, filename, canSkipChecksum, e);
         throw e;
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.info(traceString + ": returning " + nRead);
-      }
+      LOG.trace(traceFormatStr + ": returning {}",
+          buf.remaining(), block, filename, canSkipChecksum, nRead);
       return nRead;
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
@@ -490,10 +481,8 @@ class BlockReaderLocal implements BlockReader {
     }
     dataBuf.limit(dataBuf.position());
     dataBuf.position(Math.min(dataBuf.position(), slop));
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("loaded " + dataBuf.remaining() + " bytes into bounce " +
-          "buffer from offset " + oldDataPos + " of " + block);
-    }
+    LOG.trace("loaded {} bytes into bounce buffer from offset {} of {}",
+        dataBuf.remaining(), oldDataPos, block);
     return dataBuf.limit() != maxReadaheadLength;
   }
 
@@ -565,18 +554,10 @@ class BlockReaderLocal implements BlockReader {
     boolean canSkipChecksum = createNoChecksumContext();
     int nRead;
     try {
-      String traceString = null;
-      if (LOG.isTraceEnabled()) {
-        traceString = new StringBuilder().
-            append("read(arr.length=").append(arr.length).
-            append(", off=").append(off).
-            append(", len=").append(len).
-            append(", filename=").append(filename).
-            append(", block=").append(block).
-            append(", canSkipChecksum=").append(canSkipChecksum).
-            append(")").toString();
-        LOG.trace(traceString + ": starting");
-      }
+      final String traceFormatStr = "read(arr.length={}, off={}, len={}, "
+          + "filename={}, block={}, canSkipChecksum={})";
+      LOG.trace(traceFormatStr + ": starting",
+          arr.length, off, len, filename, block, canSkipChecksum);
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
           nRead = readWithoutBounceBuffer(arr, off, len);
@@ -584,14 +565,12 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
         }
       } catch (IOException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(traceString + ": I/O error", e);
-        }
+        LOG.trace(traceFormatStr + ": I/O error",
+            arr.length, off, len, filename, block, canSkipChecksum, e);
         throw e;
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(traceString + ": returning " + nRead);
-      }
+      LOG.trace(traceFormatStr + ": returning {}",
+          arr.length, off, len, filename, block, canSkipChecksum, nRead);
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
     }
@@ -634,11 +613,9 @@ class BlockReaderLocal implements BlockReader {
       dataBuf.position(dataBuf.position() + discardedFromBuf);
       remaining -= discardedFromBuf;
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("skip(n=" + n + ", block=" + block + ", filename=" + 
-        filename + "): discarded " + discardedFromBuf + " bytes from " +
-        "dataBuf and advanced dataPos by " + remaining);
-    }
+    LOG.trace("skip(n={}, block={}, filename={}): discarded {} bytes from "
+            + "dataBuf and advanced dataPos by {}",
+        n, block, filename, discardedFromBuf, remaining);
     dataPos += remaining;
     return n;
   }
@@ -653,9 +630,7 @@ class BlockReaderLocal implements BlockReader {
   public synchronized void close() throws IOException {
     if (closed) return;
     closed = true;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("close(filename=" + filename + ", block=" + block + ")");
-    }
+    LOG.trace("close(filename={}, block={})", filename, block);
     replica.unref();
     freeDataBufIfExists();
     freeChecksumBufIfExists();
@@ -705,11 +680,9 @@ class BlockReaderLocal implements BlockReader {
         (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
     if (anchor) {
       if (!createNoChecksumContext()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("can't get an mmap for " + block + " of " + filename + 
-              " since SKIP_CHECKSUMS was not given, " +
-              "we aren't skipping checksums, and the block is not mlocked.");
-        }
+        LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
+            + "given, we aren't skipping checksums, and the block is not "
+            + "mlocked.", block, filename);
         return null;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 5235287..4a1828e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -221,11 +221,9 @@ class BlockReaderLocalLegacy implements BlockReader {
       File blkfile = new File(pathinfo.getBlockPath());
       dataIn = new FileInputStream(blkfile);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("New BlockReaderLocalLegacy for file " + blkfile + " of size "
-            + blkfile.length() + " startOffset " + startOffset + " length "
-            + length + " short circuit checksum " + !skipChecksumCheck);
-      }
+      LOG.debug("New BlockReaderLocalLegacy for file {} of size {} startOffset "
+              + "{} length {} short circuit checksum {}",
+          blkfile, blkfile.length(), startOffset, length, !skipChecksumCheck);
 
       if (!skipChecksumCheck) {
         // get the metadata file
@@ -292,9 +290,7 @@ class BlockReaderLocalLegacy implements BlockReader {
       // channel for the DataNode to notify the client that the path has been
       // invalidated.  Therefore, our only option is to skip caching.
       if (pathinfo != null && !storageType.isTransient()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Cached location of block " + blk + " as " + pathinfo);
-        }
+        LOG.debug("Cached location of block {} as {}", blk, pathinfo);
         localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
       }
     } catch (IOException e) {
@@ -603,9 +599,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized int read(byte[] buf, int off, int len) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("read off " + off + " len " + len);
-    }
+    LOG.trace("read off {} len {}", off, len);
     if (!verifyChecksum) {
       return dataIn.read(buf, off, len);
     }
@@ -624,9 +618,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized long skip(long n) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("skip " + n);
-    }
+    LOG.debug("skip {}", n);
     if (n <= 0) {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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
index 92d117c..8f3df81 100644
--- 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
@@ -419,9 +419,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     final int idx = r.nextInt(localInterfaceAddrs.length);
     final SocketAddress addr = localInterfaceAddrs[idx];
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using local interface " + addr);
-    }
+    LOG.debug("Using local interface {}", addr);
     return addr;
   }
 
@@ -1216,9 +1214,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                              InetSocketAddress[] favoredNodes) throws IOException {
     checkOpen();
     final FsPermission masked = applyUMask(permission);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + masked);
-    }
+    LOG.debug("{}: masked={}", src, masked);
     final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
         src, masked, flag, createParent, replication, blockSize, progress,
         buffersize, dfsClientConf.createChecksum(checksumOpt),
@@ -1815,10 +1811,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
               smallBufferSize));
           in = new DataInputStream(pair.in);
 
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("write to " + datanodes[j] + ": "
-                + Op.BLOCK_CHECKSUM + ", block=" + block);
-          }
+          LOG.debug("write to {}: {}, block={}",
+              datanodes[j], Op.BLOCK_CHECKSUM, block);
           // get block MD5
           new Sender(out).blockChecksum(block, lb.getBlockToken());
 
@@ -1882,12 +1876,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           }
         } 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.");
-            }
+            LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
+                  + "for file {} for block {} from datanode {}. Will retry the "
+                  + "block once.",
+                src, block, datanodes[j]);
             lastRetriedIndex = i;
             done = true; // actually it's not done; but we'll retry
             i--; // repeat at i-th block
@@ -1941,9 +1933,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try {
       sock = socketFactory.createSocket();
       String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Connecting to datanode " + dnAddr);
-      }
+      LOG.debug("Connecting to datanode {}", dnAddr);
       NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
       sock.setSoTimeout(timeout);
   
@@ -2563,9 +2553,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       absPermission = applyUMask(null);
     } 
 
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + absPermission);
-    }
+    LOG.debug("{}: masked={}", src, absPermission);
     TraceScope scope = tracer.newScope("mkdir");
     try {
       return namenode.mkdirs(src, absPermission, createParent);
@@ -3061,9 +3049,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       }
     });
     HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using hedged reads; pool threads=" + num);
-    }
+    LOG.debug("Using hedged reads; pool threads={}", num);
   }
 
   ThreadPoolExecutor getHedgedReadsThreadPool() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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
index 7101753..81e8c27 100644
--- 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
@@ -315,9 +315,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if (locatedBlocks == null || refresh) {
       newInfo = dfsClient.getLocatedBlocks(src, 0);
     }
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("newInfo = " + newInfo);
-    }
+    DFSClient.LOG.debug("newInfo = {}", newInfo);
     if (newInfo == null) {
       throw new IOException("Cannot open filename " + src);
     }
@@ -383,10 +381,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           replicaNotFoundCount--;
         }
         
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
-              + datanode + " for block " + locatedblock.getBlock(), ioe);
-        }
+        DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
+              + " for block {}", datanode, locatedblock.getBlock(), ioe);
       } finally {
         if (cdp != null) {
           RPC.stopProxy(cdp);
@@ -1067,9 +1063,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     final String dnAddr =
         chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
-    }
+    DFSClient.LOG.debug("Connecting to datanode {}", dnAddr);
     InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
     return new DNAddrPair(chosenNode, targetAddr, storageType);
   }
@@ -1309,11 +1303,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             future.get();
             return;
           }
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
-                + "ms to read from " + chosenNode.info
-                + "; spawning hedged read");
-          }
+          DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
+              + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
           // Ignore this node on next go around.
           ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
@@ -1340,10 +1331,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
               .submit(getFromDataNodeCallable);
           futures.add(oneMoreRequest);
         } catch (IOException ioe) {
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Failed getting node for hedged read: "
-                + ioe.getMessage());
-          }
+          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.
@@ -1599,11 +1588,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             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);
-          }
+          DFSClient.LOG.debug("Exception while seek to {} from {} of {} from "
+              + "{}", targetPos, getCurrentBlock(), src, currentNode, e);
         }
       }
     }
@@ -1819,20 +1805,16 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     } 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);
-        }
+        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {}"
+                + " of {}; {} bytes left in block. blockPos={}; curPos={};"
+                + "curEnd={}",
+            curPos, src, length63, blockPos, curPos, 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);
-      }
+      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid going "
+              + "more than one byte past the end of the block.  blockPos={}; "
+              +" curPos={}; curEnd={}",
+          maxLength, length63, blockPos, curPos, curEnd);
     }
     // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
     int length;
@@ -1846,28 +1828,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         // 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);
-        }
+        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {} "
+            + " of {}; 31-bit MappedByteBuffer limit exceeded.  blockPos={}, "
+            + "curEnd={}", curPos, src, blockPos, 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);
-      }
+      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid 31-bit "
+          + "limit.  blockPos={}; curPos={}; curEnd={}",
+          maxLength, length, blockPos, curPos, 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.");
-      }
+      DFSClient.LOG.debug("unable to perform a zero-copy read from offset {} of"
+          + " {}; BlockReader#getClientMmap returned null.", curPos, src);
       return null;
     }
     boolean success = false;
@@ -1881,11 +1855,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       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);
-      }
+      DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
+          + "zero-copy read path.  blockEnd = {}", length, curPos, blockEnd);
       success = true;
     } finally {
       if (!success) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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
index 7a40d73..47c4b7e 100755
--- 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
@@ -190,9 +190,9 @@ public class DFSOutputStream extends FSOutputSummer
     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);
+    if (progress != null) {
+      DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
+          +"{}", src);
     }
     
     this.bytesPerChecksum = checksum.getBytesPerChecksum();
@@ -365,12 +365,9 @@ public class DFSOutputStream extends FSOutputSummer
     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);
-    }
+    DFSClient.LOG.debug("computePacketChunkSize: src={}, chunkSize={}, "
+            + "chunksPerPacket={}, packetSize={}",
+        src, chunkSize, chunksPerPacket, packetSize);
   }
 
   protected TraceScope createWriteTraceScope() {
@@ -397,14 +394,10 @@ public class DFSOutputStream extends FSOutputSummer
     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());
-      }
+      DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno={},"
+              + " src={}, packetSize={}, chunksPerPacket={}, bytesCurBlock={}",
+          currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
+          getStreamer().getBytesCurBlock());
     }
 
     currentPacket.writeChecksum(checksum, ckoff, cklen);
@@ -558,12 +551,9 @@ public class DFSOutputStream extends FSOutputSummer
         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);
-        }
+        DFSClient.LOG.debug("DFSClient flush():  bytesCurBlock={}, "
+                + "lastFlushOffset={}, createNewBlock={}",
+            getStreamer().getBytesCurBlock(), lastFlushOffset, endBlock);
         // Flush only if we haven't already flushed till this offset.
         if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
           assert getStreamer().getBytesCurBlock() > lastFlushOffset;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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 e275afb..71ce7cc 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
@@ -462,19 +462,13 @@ public class DFSUtilClient {
     InetAddress addr = targetAddr.getAddress();
     Boolean cached = localAddrMap.get(addr.getHostAddress());
     if (cached != null) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Address " + targetAddr +
-            (cached ? " is local" : " is not local"));
-      }
+      LOG.trace("Address {} is {} local", targetAddr, (cached ? "" : "not"));
       return cached;
     }
 
     boolean local = NetUtils.isLocalAddress(addr);
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Address " + targetAddr +
-          (local ? " is local" : " is not local"));
-    }
+    LOG.trace("Address {} is {} local", targetAddr, (local ? "" : "not"));
     localAddrMap.put(addr.getHostAddress(), local);
     return local;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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
index d1d8d37..4ea1f41 100644
--- 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
@@ -132,18 +132,14 @@ class DataStreamer extends Daemon {
       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);
-    }
+    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());
-    }
+    LOG.debug("Send buf size {}", sock.getSendBufferSize());
     return sock;
   }
 
@@ -484,9 +480,7 @@ class DataStreamer extends Daemon {
   }
 
   private void endBlock() {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Closing old block " + block);
-    }
+    LOG.debug("Closing old block {}", block);
     this.setName("DataStreamer for file " + src);
     closeResponder();
     closeStream();
@@ -567,15 +561,11 @@ class DataStreamer extends Daemon {
 
         // get new block from namenode.
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Allocating new block");
-          }
+          LOG.debug("Allocating new block");
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Append to block " + block);
-          }
+          LOG.debug("Append to block {}", block);
           setupPipelineForAppendOrRecovery();
           if (streamerClosed) {
             continue;
@@ -627,10 +617,7 @@ class DataStreamer extends Daemon {
           }
         }
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("DataStreamer block " + block +
-              " sending packet " + one);
-        }
+        LOG.debug("DataStreamer block {} sending packet {}", block, one);
 
         // write out data to remote datanode
         TraceScope writeScope = dfsClient.getTracer().
@@ -741,9 +728,7 @@ class DataStreamer extends Daemon {
     TraceScope scope = dfsClient.getTracer().
         newScope("waitForAckedSeqno");
     try {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Waiting for ack for: " + seqno);
-      }
+      LOG.debug("Waiting for ack for: {}", seqno);
       long begin = Time.monotonicNow();
       try {
         synchronized (dataQueue) {
@@ -955,8 +940,8 @@ class DataStreamer extends Daemon {
             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);
+          } else {
+            LOG.debug("DFSClient {}", ack);
           }
 
           long seqno = ack.getSeqno();
@@ -1176,9 +1161,7 @@ class DataStreamer extends Daemon {
   }
 
   private void addDatanode2ExistingPipeline() throws IOException {
-    if (DataTransferProtocol.LOG.isDebugEnabled()) {
-      DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
-    }
+    DataTransferProtocol.LOG.debug("lastAckedSeqno = {}", lastAckedSeqno);
       /*
        * Is data transfer necessary?  We have the following cases.
        *
@@ -1645,10 +1628,8 @@ class DataStreamer extends Daemon {
           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] + ").");
-        }
+        LOG.debug("{} was chosen by name node (favored={}).",
+            nodes[i].getXferAddrWithHostname(), pinnings[i]);
       }
       if (shouldLog && !favoredSet.isEmpty()) {
         // There is one or more favored nodes that were not allocated.
@@ -1787,9 +1768,7 @@ class DataStreamer extends Daemon {
       packet.addTraceParent(Tracer.getCurrentSpanId());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Queued packet " + packet.getSeqno());
-      }
+      LOG.debug("Queued packet {}", packet.getSeqno());
       dataQueue.notifyAll();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index 15a5bee..017be9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -250,9 +250,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       PacketHeader header = new PacketHeader();
       header.readFields(in);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("DFSClient readChunk got header " + header);
-      }
+      LOG.debug("DFSClient readChunk got header {}", header);
 
       // Sanity check the lengths
       if (!header.sanityCheck(lastSeqNo)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 7a7932d..ca31e67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -135,14 +135,9 @@ public class RemoteBlockReader2  implements BlockReader {
   @Override
   public synchronized int read(byte[] buf, int off, int len) 
                                throws IOException {
-
-    UUID randomId = null;
-    if (LOG.isTraceEnabled()) {
-      randomId = UUID.randomUUID();
-      LOG.trace(String.format("Starting read #%s file %s from datanode %s",
-        randomId.toString(), this.filename,
-        this.datanodeID.getHostName()));
-    }
+    UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
+    LOG.trace("Starting read #{} file {} from datanode {}",
+        randomId, filename, datanodeID.getHostName());
 
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
       TraceScope scope = tracer.newScope(
@@ -154,9 +149,7 @@ public class RemoteBlockReader2  implements BlockReader {
       }
     }
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("Finishing read #" + randomId));
-    }
+    LOG.trace("Finishing read #{}", randomId);
 
     if (curDataSlice.remaining() == 0) {
       // we're at EOF now
@@ -203,9 +196,7 @@ public class RemoteBlockReader2  implements BlockReader {
     curDataSlice = packetReceiver.getDataSlice();
     assert curDataSlice.capacity() == curHeader.getDataLen();
     
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("DFSClient readNextPacket got header " + curHeader);
-    }
+    LOG.trace("DFSClient readNextPacket got header {}", curHeader);
 
     // Sanity check the lengths
     if (!curHeader.sanityCheck(lastSeqNo)) {
@@ -276,10 +267,8 @@ public class RemoteBlockReader2  implements BlockReader {
   }
 
   private void readTrailingEmptyPacket() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Reading empty packet at end of read");
-    }
-    
+    LOG.trace("Reading empty packet at end of read");
+
     packetReceiver.receiveNextPacket(in);
 
     PacketHeader trailer = packetReceiver.getHeader();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/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
index c3d2cfc..8457d65 100644
--- 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
@@ -308,10 +308,7 @@ public class LeaseRenewer {
               }
               LeaseRenewer.this.run(id);
             } catch(InterruptedException e) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
-                    + " is interrupted.", e);
-              }
+              LOG.debug("LeaseRenewer is interrupted.", e);
             } finally {
               synchronized(LeaseRenewer.this) {
                 Factory.INSTANCE.remove(LeaseRenewer.this);
@@ -399,9 +396,7 @@ public class LeaseRenewer {
     }
 
     if (daemonCopy != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Wait for lease checker to terminate");
-      }
+      LOG.debug("Wait for lease checker to terminate");
       daemonCopy.join();
     }
   }
@@ -424,16 +419,11 @@ public class LeaseRenewer {
       //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);
-          }
+          LOG.debug("Did not renew lease for client {}", c);
           continue;
         }
         previousName = c.getClientName();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Lease renewed for client " + previousName);
-        }
+        LOG.debug("Lease renewed for client {}", previousName);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
index c4093b1..e6709d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
@@ -147,11 +147,9 @@ public class PacketReceiver implements Closeable {
       throw new IOException("Invalid header length " + headerLen);
     }
     
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("readNextPacket: dataPlusChecksumLen = " + dataPlusChecksumLen +
-          " headerLen = " + headerLen);
-    }
-    
+    LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
+        dataPlusChecksumLen, headerLen);
+
     // Sanity check the buffer size so we don't allocate too much memory
     // and OOME.
     int totalLen = payloadLen + headerLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index e856211..d2bc348 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -73,10 +73,8 @@ public class Sender implements DataTransferProtocol {
 
   private static void send(final DataOutputStream out, final Op opcode,
       final Message proto) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Sending DataTransferOp " + proto.getClass().getSimpleName()
-          + ": " + proto);
-    }
+    LOG.trace("Sending DataTransferOp {}: {}",
+        proto.getClass().getSimpleName(), proto);
     op(out, opcode);
     proto.writeDelimitedTo(out);
     out.flush();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
index 256caff..006d304 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
@@ -332,11 +332,9 @@ public final class DataTransferSaslUtil {
   public static IOStreamPair createStreamPair(Configuration conf,
       CipherOption cipherOption, OutputStream out, InputStream in, 
       boolean isServer) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Creating IOStreamPair of CryptoInputStream and " +
-          "CryptoOutputStream.");
-    }
-    CryptoCodec codec = CryptoCodec.getInstance(conf, 
+    LOG.debug("Creating IOStreamPair of CryptoInputStream and "
+        + "CryptoOutputStream.");
+    CryptoCodec codec = CryptoCodec.getInstance(conf,
         cipherOption.getCipherSuite());
     byte[] inKey = cipherOption.getInKey();
     byte[] inIv = cipherOption.getInIv();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index f764275..24e1dd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -130,9 +130,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
-    }
+    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
     rpcProxy = createClientDatanodeProtocolProxy(addr,
         UserGroupInformation.getCurrentUser(), conf,
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
@@ -143,10 +141,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
       boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
-    }
-    
+    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
+
     // Since we're creating a new UserGroupInformation here, we know that no
     // future RPC proxies will be able to re-use the same connection. And
     // usages of this proxy tend to be one-off calls.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
index f70398a..4ffc108 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
@@ -129,18 +129,13 @@ public class DfsClientShmManager implements Closeable {
       ShmId shmId = shm.getShmId();
       Slot slot = shm.allocAndRegisterSlot(blockId);
       if (shm.isFull()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": pulled the last slot " + slot.getSlotIdx() +
-              " out of " + shm);
-        }
+        LOG.trace("{}: pulled the last slot {} out of {}",
+            this, slot.getSlotIdx(), shm);
         DfsClientShm removedShm = notFull.remove(shmId);
         Preconditions.checkState(removedShm == shm);
         full.put(shmId, shm);
       } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": pulled slot " + slot.getSlotIdx() +
-              " out of " + shm);
-        }
+        LOG.trace("{}: pulled slot {} out of {}", this, slot.getSlotIdx(), shm);
       }
       return slot;
     }
@@ -187,9 +182,7 @@ public class DfsClientShmManager implements Closeable {
           DfsClientShm shm = 
               new DfsClientShm(PBHelperClient.convert(resp.getId()),
                   fis[0], this, peer);
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": createNewShm: created " + shm);
-          }
+          LOG.trace("{}: createNewShm: created {}", this, shm);
           return shm;
         } finally {
           try {
@@ -234,15 +227,11 @@ public class DfsClientShmManager implements Closeable {
         String clientName, ExtendedBlockId blockId) throws IOException {
       while (true) {
         if (closed) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": the DfsClientShmManager has been closed.");
-          }
+          LOG.trace("{}: the DfsClientShmManager has been closed.", this);
           return null;
         }
         if (disabled) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": shared memory segment access is disabled.");
-          }
+          LOG.trace("{}: shared memory segment access is disabled.", this);
           return null;
         }
         // Try to use an existing slot.
@@ -253,9 +242,7 @@ public class DfsClientShmManager implements Closeable {
         // There are no free slots.  If someone is loading more slots, wait
         // for that to finish.
         if (loading) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": waiting for loading to finish...");
-          }
+          LOG.trace("{}: waiting for loading to finish...", this);
           finishedLoading.awaitUninterruptibly();
         } else {
           // Otherwise, load the slot ourselves.
@@ -282,11 +269,9 @@ public class DfsClientShmManager implements Closeable {
             // fired and marked the shm as disconnected.  In this case, we
             // obviously don't want to add the SharedMemorySegment to our list
             // of valid not-full segments.
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(this + ": the UNIX domain socket associated with " +
-                  "this short-circuit memory closed before we could make " +
-                  "use of the shm.");
-            }
+            LOG.debug("{}: the UNIX domain socket associated with this "
+                + "short-circuit memory closed before we could make use of "
+                + "the shm.", this);
           } else {
             notFull.put(shm.getShmId(), shm);
           }
@@ -309,9 +294,7 @@ public class DfsClientShmManager implements Closeable {
         Preconditions.checkState(!full.containsKey(shm.getShmId()));
         Preconditions.checkState(!notFull.containsKey(shm.getShmId()));
         if (shm.isEmpty()) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": freeing empty stale " + shm);
-          }
+          LOG.trace("{}: freeing empty stale {}", this, shm);
           shm.free();
         }
       } else {
@@ -336,10 +319,8 @@ public class DfsClientShmManager implements Closeable {
           // lowest ID, but it could still occur.  In most workloads,
           // fragmentation should not be a major concern, since it doesn't impact
           // peak file descriptor usage or the speed of allocation.
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": shutting down UNIX domain socket for " +
-                "empty " + shm);
-          }
+          LOG.trace("{}: shutting down UNIX domain socket for empty {}",
+              this, shm);
           shutdown(shm);
         } else {
           notFull.put(shmId, shm);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
index 52c1a6e..07f5064 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
@@ -103,9 +103,7 @@ public class ShortCircuitCache implements Closeable {
         if (ShortCircuitCache.this.closed) return;
         long curMs = Time.monotonicNow();
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(this + ": cache cleaner running at " + curMs);
-        }
+        LOG.debug("{}: cache cleaner running at {}", this, curMs);
 
         int numDemoted = demoteOldEvictableMmaped(curMs);
         int numPurged = 0;
@@ -127,11 +125,9 @@ public class ShortCircuitCache implements Closeable {
           numPurged++;
         }
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(this + ": finishing cache cleaner run started at " +
-            curMs + ".  Demoted " + numDemoted + " mmapped replicas; " +
-            "purged " + numPurged + " replicas.");
-        }
+        LOG.debug("{}: finishing cache cleaner run started at {}. Demoted {} "
+            + "mmapped replicas; purged {} replicas.",
+            this, curMs, numDemoted, numPurged);
       } finally {
         ShortCircuitCache.this.lock.unlock();
       }
@@ -186,9 +182,7 @@ public class ShortCircuitCache implements Closeable {
 
     @Override
     public void run() {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(ShortCircuitCache.this + ": about to release " + slot);
-      }
+      LOG.trace("{}: about to release {}", ShortCircuitCache.this, slot);
       final DfsClientShm shm = (DfsClientShm)slot.getShm();
       final DomainSocket shmSock = shm.getPeer().getDomainSocket();
       final String path = shmSock.getPath();
@@ -205,9 +199,7 @@ public class ShortCircuitCache implements Closeable {
           String error = resp.hasError() ? resp.getError() : "(unknown)";
           throw new IOException(resp.getStatus().toString() + ": " + error);
         }
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(ShortCircuitCache.this + ": released " + slot);
-        }
+        LOG.trace("{}: released {}", this, slot);
         success = true;
       } catch (IOException e) {
         LOG.error(ShortCircuitCache.this + ": failed to release " +
@@ -433,9 +425,7 @@ public class ShortCircuitCache implements Closeable {
           purgeReason = "purging replica because it is stale.";
         }
         if (purgeReason != null) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(this + ": " + purgeReason);
-          }
+          LOG.debug("{}: {}", this, purgeReason);
           purge(replica);
         }
       }
@@ -677,10 +667,8 @@ public class ShortCircuitCache implements Closeable {
       ShortCircuitReplicaInfo info = null;
       do {
         if (closed) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": can't fetchOrCreate " + key +
-                " because the cache is closed.");
-          }
+          LOG.trace("{}: can't fethchOrCreate {} because the cache is closed.",
+              this, key);
           return null;
         }
         Waitable<ShortCircuitReplicaInfo> waitable = replicaInfoMap.get(key);
@@ -688,9 +676,7 @@ public class ShortCircuitCache implements Closeable {
           try {
             info = fetch(key, waitable);
           } catch (RetriableException e) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(this + ": retrying " + e.getMessage());
-            }
+            LOG.debug("{}: retrying {}", this, e.getMessage());
             continue;
           }
         }
@@ -721,9 +707,7 @@ public class ShortCircuitCache implements Closeable {
     // ShortCircuitReplica.  So we simply wait for it to complete.
     ShortCircuitReplicaInfo info;
     try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": found waitable for " + key);
-      }
+      LOG.trace("{}: found waitable for {}", this, key);
       info = waitable.await();
     } catch (InterruptedException e) {
       LOG.info(this + ": interrupted while waiting for " + key);
@@ -765,9 +749,7 @@ public class ShortCircuitCache implements Closeable {
     // Handle loading a new replica.
     ShortCircuitReplicaInfo info = null;
     try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": loading " + key);
-      }
+      LOG.trace("{}: loading {}", this, key);
       info = creator.createShortCircuitReplicaInfo();
     } catch (RuntimeException e) {
       LOG.warn(this + ": failed to load " + key, e);
@@ -777,9 +759,7 @@ public class ShortCircuitCache implements Closeable {
     try {
       if (info.getReplica() != null) {
         // On success, make sure the cache cleaner thread is running.
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": successfully loaded " + info.getReplica());
-        }
+        LOG.trace("{}: successfully loaded {}", this, info.getReplica());
         startCacheCleanerThreadIfNeeded();
         // Note: new ShortCircuitReplicas start with a refCount of 2,
         // indicating that both this cache and whoever requested the 
@@ -811,10 +791,8 @@ public class ShortCircuitCache implements Closeable {
           cleanerExecutor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
               TimeUnit.MILLISECONDS);
       cacheCleaner.setFuture(future);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(this + ": starting cache cleaner thread which will run " +
-          "every " + rateMs + " ms");
-      }
+      LOG.debug("{}: starting cache cleaner thread which will run every {} ms",
+          this, rateMs);
     }
   }
 
@@ -832,17 +810,12 @@ public class ShortCircuitCache implements Closeable {
           long lastAttemptTimeMs = (Long)replica.mmapData;
           long delta = Time.monotonicNow() - lastAttemptTimeMs;
           if (delta < mmapRetryTimeoutMs) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace(this + ": can't create client mmap for " +
-                  replica + " because we failed to " +
-                  "create one just " + delta + "ms ago.");
-            }
+            LOG.trace("{}: can't create client mmap for {} because we failed to"
+                + " create one just {}ms ago.", this, replica, delta);
             return null;
           }
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": retrying client mmap for " + replica +
-                ", " + delta + " ms after the previous failure.");
-          }
+          LOG.trace("{}: retrying client mmap for {}, {} ms after the previous "
+              + "failure.", this, replica, delta);
         } else if (replica.mmapData instanceof Condition) {
           Condition cond = (Condition)replica.mmapData;
           cond.awaitUninterruptibly();
@@ -965,38 +938,10 @@ public class ShortCircuitCache implements Closeable {
           }
         }
       }
-      if (LOG.isDebugEnabled()) {
-        StringBuilder builder = new StringBuilder();
-        builder.append("visiting ").append(visitor.getClass().getName()).
-            append("with outstandingMmapCount=").append(outstandingMmapCount).
-            append(", replicas=");
-        String prefix = "";
-        for (Entry<ExtendedBlockId, ShortCircuitReplica> entry : replicas.entrySet()) {
-          builder.append(prefix).append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", failedLoads=");
-        for (Entry<ExtendedBlockId, InvalidToken> entry : failedLoads.entrySet()) {
-          builder.append(prefix).append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", evictable=");
-        for (Entry<Long, ShortCircuitReplica> entry : evictable.entrySet()) {
-          builder.append(prefix).append(entry.getKey()).
-              append(":").append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", evictableMmapped=");
-        for (Entry<Long, ShortCircuitReplica> entry : evictableMmapped.entrySet()) {
-          builder.append(prefix).append(entry.getKey()).
-              append(":").append(entry.getValue());
-          prefix = ",";
-        }
-        LOG.debug(builder.toString());
-      }
+      LOG.debug("visiting {} with outstandingMmapCount={}, replicas={}, "
+          + "failedLoads={}, evictable={}, evictableMmapped={}",
+          visitor.getClass().getName(), outstandingMmapCount, replicas,
+          failedLoads, evictable, evictableMmapped);
       visitor.visit(outstandingMmapCount, replicas, failedLoads,
             evictable, evictableMmapped);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
index 37566e2..38cf22b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
@@ -154,25 +154,19 @@ public class ShortCircuitReplica {
       // Check staleness by looking at the shared memory area we use to
       // communicate with the DataNode.
       boolean stale = !slot.isValid();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": checked shared memory segment.  isStale=" + stale);
-      }
+      LOG.trace("{}: checked shared memory segment.  isStale={}", this, stale);
       return stale;
     } else {
       // Fall back to old, time-based staleness method.
       long deltaMs = Time.monotonicNow() - creationTimeMs;
       long staleThresholdMs = cache.getStaleThresholdMs();
       if (deltaMs > staleThresholdMs) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + " is stale because it's " + deltaMs +
-              " ms old, and staleThresholdMs = " + staleThresholdMs);
-        }
+        LOG.trace("{} is stale because it's {} ms old and staleThreadholdMS={}",
+            this, deltaMs, staleThresholdMs);
         return true;
       } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + " is not stale because it's only " + deltaMs +
-              " ms old, and staleThresholdMs = " + staleThresholdMs);
-        }
+        LOG.trace("{} is not stale because it's only {} ms old "
+            + "and staleThresholdMs={}",  this, deltaMs, staleThresholdMs);
         return false;
       }
     }
@@ -194,13 +188,8 @@ public class ShortCircuitReplica {
       return false;
     }
     boolean result = slot.addAnchor();
-    if (LOG.isTraceEnabled()) {
-      if (result) {
-        LOG.trace(this + ": added no-checksum anchor to slot " + slot);
-      } else {
-        LOG.trace(this + ": could not add no-checksum anchor to slot " + slot);
-      }
-    }
+    LOG.trace("{}: {} no-checksum anchor to slot {}",
+        this, result ? "added" : "could not add", slot);
     return result;
   }
 
@@ -263,9 +252,7 @@ public class ShortCircuitReplica {
         suffix += "  scheduling " + slot + " for later release.";
       }
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("closed " + this + suffix);
-    }
+    LOG.trace("closed {}{}", this, suffix);
   }
 
   public FileInputStream getDataStream() {
@@ -293,9 +280,7 @@ public class ShortCircuitReplica {
       FileChannel channel = dataStream.getChannel();
       MappedByteBuffer mmap = channel.map(MapMode.READ_ONLY, 0, 
           Math.min(Integer.MAX_VALUE, channel.size()));
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": created mmap of size " + channel.size());
-      }
+      LOG.trace("{}: created mmap of size {}", this, channel.size());
       return mmap;
     } catch (IOException e) {
       LOG.warn(this + ": mmap error", e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
index 78325a3..fa40c15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
@@ -484,13 +484,9 @@ public class ShortCircuitShm {
         POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, mmappedLength);
     this.slots = new Slot[mmappedLength / BYTES_PER_SLOT];
     this.allocatedSlots = new BitSet(slots.length);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("creating " + this.getClass().getSimpleName() +
-          "(shmId=" + shmId +
-          ", mmappedLength=" + mmappedLength +
-          ", baseAddress=" + String.format("%x", baseAddress) +
-          ", slots.length=" + slots.length + ")");
-    }
+    LOG.trace("creating {}(shmId={}, mmappedLength={}, baseAddress={}, "
+        + "slots.length={})", this.getClass().getSimpleName(), shmId,
+        mmappedLength, String.format("%x", baseAddress), slots.length);
   }
 
   public final ShmId getShmId() {
@@ -615,9 +611,7 @@ public class ShortCircuitShm {
         "tried to unregister slot " + slotIdx + ", which was not registered.");
     allocatedSlots.set(slotIdx, false);
     slots[slotIdx] = null;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": unregisterSlot " + slotIdx);
-    }
+    LOG.trace("{}: unregisterSlot {}", this, slotIdx);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
index a9adb7e..e361252 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
@@ -36,18 +36,6 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public abstract class ByteArrayManager {
   static final Logger LOG = LoggerFactory.getLogger(ByteArrayManager.class);
-  private static final ThreadLocal<StringBuilder> DEBUG_MESSAGE =
-      new ThreadLocal<StringBuilder>() {
-    protected StringBuilder initialValue() {
-      return new StringBuilder();
-    }
-  };
-
-  private static void logDebugMessage() {
-    final StringBuilder b = DEBUG_MESSAGE.get();
-    LOG.debug(b.toString());
-    b.setLength(0);
-  }
 
   static final int MIN_ARRAY_LENGTH = 32;
   static final byte[] EMPTY_BYTE_ARRAY = {};
@@ -160,27 +148,18 @@ public abstract class ByteArrayManager {
      * via the {@link FixedLengthManager#recycle(byte[])} method.
      */
     synchronized byte[] allocate() throws InterruptedException {
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", ").append(this);
-      }
+      LOG.debug(", {}", this);
       for(; numAllocated >= maxAllocated;) {
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(": wait ...");
-          logDebugMessage();
-        }
+        LOG.debug(": wait ...");
 
         wait();
 
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append("wake up: ").append(this);
-        }
+        LOG.debug("wake up: {}", this);
       }
       numAllocated++;
 
       final byte[] array = freeQueue.poll();
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", recycled? ").append(array != null);
-      }
+      LOG.debug(", recycled? {}", array != null);
       return array != null? array : new byte[byteArrayLength];
     }
 
@@ -194,9 +173,7 @@ public abstract class ByteArrayManager {
     synchronized int recycle(byte[] array) {
       Preconditions.checkNotNull(array);
       Preconditions.checkArgument(array.length == byteArrayLength);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", ").append(this);
-      }
+      LOG.debug(", {}", this);
 
       notify();
       numAllocated--;
@@ -207,9 +184,7 @@ public abstract class ByteArrayManager {
       }
 
       if (freeQueue.size() < maxAllocated - numAllocated) {
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(", freeQueue.offer");
-        }
+        LOG.debug(", freeQueue.offer");
         freeQueue.offer(array);
       }
       return freeQueue.size();
@@ -349,9 +324,7 @@ public abstract class ByteArrayManager {
     public byte[] newByteArray(final int arrayLength)
         throws InterruptedException {
       Preconditions.checkArgument(arrayLength >= 0);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append("allocate(").append(arrayLength).append(")");
-      }
+      LOG.debug("allocate({})", arrayLength);
 
       final byte[] array;
       if (arrayLength == 0) {
@@ -365,18 +338,12 @@ public abstract class ByteArrayManager {
         final FixedLengthManager manager =
             managers.get(powerOfTwo, aboveThreshold);
 
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(": count=").append(count)
-              .append(aboveThreshold? ", aboveThreshold": ", belowThreshold");
-        }
+        LOG.debug(": count={}, {}Threshold", count,
+            aboveThreshold ? "above" : "below");
         array = manager != null? manager.allocate(): new byte[powerOfTwo];
       }
 
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", return byte[")
-            .append(array.length).append("]");
-        logDebugMessage();
-      }
+      LOG.debug(", return byte[{}]", array.length);
       return array;
     }
 
@@ -391,10 +358,7 @@ public abstract class ByteArrayManager {
     @Override
     public int release(final byte[] array) {
       Preconditions.checkNotNull(array);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get()
-            .append("recycle: array.length=").append(array.length);
-      }
+      LOG.debug("recycle: array.length={}", array.length);
 
       final int freeQueueSize;
       if (array.length == 0) {
@@ -404,10 +368,7 @@ public abstract class ByteArrayManager {
         freeQueueSize = manager == null? -1: manager.recycle(array);
       }
 
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", freeQueueSize=").append(freeQueueSize);
-        logDebugMessage();
-      }
+      LOG.debug(", freeQueueSize={}", freeQueueSize);
       return freeQueueSize;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
index a864d37..870103e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
@@ -134,9 +134,7 @@ final class TokenAspect<T extends FileSystem & Renewable> {
       if (token != null) {
         fs.setDelegationToken(token);
         addRenewAction(fs);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Created new DT for {}", token.getService());
-        }
+        LOG.debug("Created new DT for {}", token.getService());
       }
       hasInitedToken = true;
     }
@@ -149,9 +147,7 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   synchronized void initDelegationToken(UserGroupInformation ugi) {
     Token<?> token = selectDelegationToken(ugi);
     if (token != null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Found existing DT for {}", token.getService());
-      }
+      LOG.debug("Found existing DT for {}", token.getService());
       fs.setDelegationToken(token);
       hasInitedToken = true;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39285e6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
index 4c23241..be5f17d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
@@ -182,9 +182,7 @@ public class URLConnectionFactory {
   public URLConnection openConnection(URL url, boolean isSpnego)
       throws IOException, AuthenticationException {
     if (isSpnego) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("open AuthenticatedURL connection {}", url);
-      }
+      LOG.debug("open AuthenticatedURL connection {}", url);
       UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
       final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
       return new AuthenticatedURL(new KerberosUgiAuthenticator(),


[24/58] [abbrv] hadoop git commit: HDFS-9147. Fix the setting of visibleLength in ExternalBlockReader. (Colin P. McCabe via Lei (Eddy) Xu)

Posted by zh...@apache.org.
HDFS-9147. Fix the setting of visibleLength in ExternalBlockReader.  (Colin P. McCabe via Lei (Eddy) Xu)


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

Branch: refs/heads/HDFS-7285
Commit: e5992ef4df63fbc6a6b8e357b32c647e7837c662
Parents: 50741cb
Author: Lei Xu <le...@apache.org>
Authored: Mon Sep 28 13:39:00 2015 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Mon Sep 28 13:39:00 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/BlockReaderFactory.java    |  5 +++--
 .../org/apache/hadoop/hdfs/ExternalBlockReader.java   | 14 +++++++-------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt           |  3 +++
 .../apache/hadoop/hdfs/TestExternalBlockReader.java   | 11 +++++++----
 4 files changed, 20 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5992ef4/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
index 273d8cf..4f37090 100644
--- 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
@@ -392,6 +392,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         Constructor<? extends ReplicaAccessorBuilder> ctor =
             cls.getConstructor();
         ReplicaAccessorBuilder builder = ctor.newInstance();
+        long visibleLength = startOffset + length;
         ReplicaAccessor accessor = builder.
             setAllowShortCircuitReads(allowShortCircuitLocalReads).
             setBlock(block.getBlockId(), block.getBlockPoolId()).
@@ -401,7 +402,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             setConfiguration(configuration).
             setFileName(fileName).
             setVerifyChecksum(verifyChecksum).
-            setVisibleLength(length).
+            setVisibleLength(visibleLength).
             build();
         if (accessor == null) {
           if (LOG.isTraceEnabled()) {
@@ -409,7 +410,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
                 cls.getName());
           }
         } else {
-          return new ExternalBlockReader(accessor, length, startOffset);
+          return new ExternalBlockReader(accessor, visibleLength, startOffset);
         }
       } catch (Throwable t) {
         LOG.warn("Failed to construct new object of type " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5992ef4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
index 3711a9d..2eb9d52 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
@@ -68,7 +68,8 @@ public final class ExternalBlockReader implements BlockReader {
     if (n <= 0) {
       return 0;
     }
-    // You can't skip past the end of the replica.
+    // You can't skip past the last offset that we want to read with this
+    // block reader.
     long oldPos = pos;
     pos += n;
     if (pos > visibleLength) {
@@ -79,12 +80,11 @@ public final class ExternalBlockReader implements BlockReader {
 
   @Override
   public int available() throws IOException {
-    // We return the amount of bytes that we haven't read yet from the
-    // replica, based on our current position.  Some of the other block
-    // readers return a shorter length than that.  The only advantage to
-    // returning a shorter length is that the DFSInputStream will
-    // trash your block reader and create a new one if someone tries to
-    // seek() beyond the available() region.
+    // We return the amount of bytes between the current offset and the visible
+    // length.  Some of the other block readers return a shorter length than
+    // that.  The only advantage to returning a shorter length is that the
+    // DFSInputStream will trash your block reader and create a new one if
+    // someone tries to seek() beyond the available() region.
     long diff = visibleLength - pos;
     if (diff > Integer.MAX_VALUE) {
       return Integer.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5992ef4/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 5e1fd92..3daf8d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1444,6 +1444,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9107. Prevent NN's unrecoverable death spiral after full GC (Daryn
     Sharp via Colin P. McCabe)
 
+    HDFS-9147. Fix the setting of visibleLength in ExternalBlockReader. (Colin
+    P. McCabe via Lei (Eddy) Xu)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5992ef4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java
index e039145..2c36baa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java
@@ -190,7 +190,7 @@ public class TestExternalBlockReader {
             "than 0 at " + pos);
         return 0;
       }
-      int i = 0, nread = 0, ipos;
+      int i = off, nread = 0, ipos;
       for (ipos = (int)pos;
            (ipos < contents.length) && (nread < len);
            ipos++) {
@@ -280,7 +280,10 @@ public class TestExternalBlockReader {
       HdfsDataInputStream stream =
           (HdfsDataInputStream)dfs.open(new Path("/a"));
       byte buf[] = new byte[TEST_LENGTH];
-      IOUtils.readFully(stream, buf, 0, TEST_LENGTH);
+      stream.seek(1000);
+      IOUtils.readFully(stream, buf, 1000, TEST_LENGTH - 1000);
+      stream.seek(0);
+      IOUtils.readFully(stream, buf, 0, 1000);
       byte expected[] = DFSTestUtil.
           calculateFileContentsFromSeed(SEED, TEST_LENGTH);
       ReadStatistics stats = stream.getReadStatistics();
@@ -293,7 +296,7 @@ public class TestExternalBlockReader {
       Assert.assertNotNull(block);
       LinkedList<SyntheticReplicaAccessor> accessorList = accessors.get(uuid);
       Assert.assertNotNull(accessorList);
-      Assert.assertEquals(2, accessorList.size());
+      Assert.assertEquals(3, accessorList.size());
       SyntheticReplicaAccessor accessor = accessorList.get(0);
       Assert.assertTrue(accessor.builder.allowShortCircuit);
       Assert.assertEquals(block.getBlockPoolId(),
@@ -307,7 +310,7 @@ public class TestExternalBlockReader {
           accessor.getGenerationStamp());
       Assert.assertTrue(accessor.builder.verifyChecksum);
       Assert.assertEquals(1024L, accessor.builder.visibleLength);
-      Assert.assertEquals(1024L, accessor.totalRead);
+      Assert.assertEquals(24L, accessor.totalRead);
       Assert.assertEquals("", accessor.getError());
       Assert.assertEquals(1, accessor.numCloses);
       byte[] tempBuf = new byte[5];