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 ji...@apache.org on 2015/05/05 01:43:34 UTC

[30/33] hadoop git commit: HDFS-8237. Move all protocol classes used by ClientProtocol to hdfs-client. Contributed by Haohui Mai.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
deleted file mode 100644
index f6b3c34..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
+++ /dev/null
@@ -1,358 +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.util.Date;
-
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtil;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Describes a path-based cache directive.
- */
-@InterfaceStability.Evolving
-@InterfaceAudience.Public
-public class CacheDirectiveInfo {
-  /**
-   * A builder for creating new CacheDirectiveInfo instances.
-   */
-  public static class Builder {
-    private Long id;
-    private Path path;
-    private Short replication;
-    private String pool;
-    private Expiration expiration;
-
-    /**
-     * Builds a new CacheDirectiveInfo populated with the set properties.
-     * 
-     * @return New CacheDirectiveInfo.
-     */
-    public CacheDirectiveInfo build() {
-      return new CacheDirectiveInfo(id, path, replication, pool, expiration);
-    }
-
-    /**
-     * Creates an empty builder.
-     */
-    public Builder() {
-    }
-
-    /**
-     * Creates a builder with all elements set to the same values as the
-     * given CacheDirectiveInfo.
-     */
-    public Builder(CacheDirectiveInfo directive) {
-      this.id = directive.getId();
-      this.path = directive.getPath();
-      this.replication = directive.getReplication();
-      this.pool = directive.getPool();
-      this.expiration = directive.getExpiration();
-    }
-
-    /**
-     * Sets the id used in this request.
-     * 
-     * @param id The id used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setId(Long id) {
-      this.id = id;
-      return this;
-    }
-
-    /**
-     * Sets the path used in this request.
-     * 
-     * @param path The path used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setPath(Path path) {
-      this.path = path;
-      return this;
-    }
-
-    /**
-     * Sets the replication used in this request.
-     * 
-     * @param replication The replication used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setReplication(Short replication) {
-      this.replication = replication;
-      return this;
-    }
-
-    /**
-     * Sets the pool used in this request.
-     * 
-     * @param pool The pool used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setPool(String pool) {
-      this.pool = pool;
-      return this;
-    }
-
-    /**
-     * Sets when the CacheDirective should expire. A
-     * {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
-     * relative expiration time.
-     * 
-     * @param expiration when this CacheDirective should expire
-     * @return This builder, for call chaining
-     */
-    public Builder setExpiration(Expiration expiration) {
-      this.expiration = expiration;
-      return this;
-    }
-  }
-
-  /**
-   * Denotes a relative or absolute expiration time for a CacheDirective. Use
-   * factory methods {@link CacheDirectiveInfo.Expiration#newAbsolute(Date)} and
-   * {@link CacheDirectiveInfo.Expiration#newRelative(long)} to create an
-   * Expiration.
-   * <p>
-   * In either case, the server-side clock is used to determine when a
-   * CacheDirective expires.
-   */
-  public static class Expiration {
-
-    /**
-     * The maximum value we accept for a relative expiry.
-     */
-    public static final long MAX_RELATIVE_EXPIRY_MS =
-        Long.MAX_VALUE / 4; // This helps prevent weird overflow bugs
-
-    /**
-     * An relative Expiration that never expires.
-     */
-    public static final Expiration NEVER = newRelative(MAX_RELATIVE_EXPIRY_MS);
-
-    /**
-     * Create a new relative Expiration.
-     * <p>
-     * Use {@link Expiration#NEVER} to indicate an Expiration that never
-     * expires.
-     * 
-     * @param ms how long until the CacheDirective expires, in milliseconds
-     * @return A relative Expiration
-     */
-    public static Expiration newRelative(long ms) {
-      return new Expiration(ms, true);
-    }
-
-    /**
-     * Create a new absolute Expiration.
-     * <p>
-     * Use {@link Expiration#NEVER} to indicate an Expiration that never
-     * expires.
-     * 
-     * @param date when the CacheDirective expires
-     * @return An absolute Expiration
-     */
-    public static Expiration newAbsolute(Date date) {
-      return new Expiration(date.getTime(), false);
-    }
-
-    /**
-     * Create a new absolute Expiration.
-     * <p>
-     * Use {@link Expiration#NEVER} to indicate an Expiration that never
-     * expires.
-     * 
-     * @param ms when the CacheDirective expires, in milliseconds since the Unix
-     *          epoch.
-     * @return An absolute Expiration
-     */
-    public static Expiration newAbsolute(long ms) {
-      return new Expiration(ms, false);
-    }
-
-    private final long ms;
-    private final boolean isRelative;
-
-    private Expiration(long ms, boolean isRelative) {
-      if (isRelative) {
-        Preconditions.checkArgument(ms <= MAX_RELATIVE_EXPIRY_MS,
-            "Expiration time is too far in the future!");
-      }
-      this.ms = ms;
-      this.isRelative = isRelative;
-    }
-
-    /**
-     * @return true if Expiration was specified as a relative duration, false if
-     *         specified as an absolute time.
-     */
-    public boolean isRelative() {
-      return isRelative;
-    }
-
-    /**
-     * @return The raw underlying millisecond value, either a relative duration
-     *         or an absolute time as milliseconds since the Unix epoch.
-     */
-    public long getMillis() {
-      return ms;
-    }
-
-    /**
-     * @return Expiration time as a {@link Date} object. This converts a
-     *         relative Expiration into an absolute Date based on the local
-     *         clock.
-     */
-    public Date getAbsoluteDate() {
-      return new Date(getAbsoluteMillis());
-    }
-
-    /**
-     * @return Expiration time in milliseconds from the Unix epoch. This
-     *         converts a relative Expiration into an absolute time based on the
-     *         local clock.
-     */
-    public long getAbsoluteMillis() {
-      if (!isRelative) {
-        return ms;
-      } else {
-        return new Date().getTime() + ms;
-      }
-    }
-
-    @Override
-    public String toString() {
-      if (isRelative) {
-        return DFSUtil.durationToString(ms);
-      }
-      return DFSUtil.dateToIso8601String(new Date(ms));
-    }
-  }
-
-  private final Long id;
-  private final Path path;
-  private final Short replication;
-  private final String pool;
-  private final Expiration expiration;
-
-  CacheDirectiveInfo(Long id, Path path, Short replication, String pool,
-      Expiration expiration) {
-    this.id = id;
-    this.path = path;
-    this.replication = replication;
-    this.pool = pool;
-    this.expiration = expiration;
-  }
-
-  /**
-   * @return The ID of this directive.
-   */
-  public Long getId() {
-    return id;
-  }
-
-  /**
-   * @return The path used in this request.
-   */
-  public Path getPath() {
-    return path;
-  }
-
-  /**
-   * @return The number of times the block should be cached.
-   */
-  public Short getReplication() {
-    return replication;
-  }
-
-  /**
-   * @return The pool used in this request.
-   */
-  public String getPool() {
-    return pool;
-  }
-
-  /**
-   * @return When this directive expires.
-   */
-  public Expiration getExpiration() {
-    return expiration;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o == null) {
-      return false;
-    }
-    if (getClass() != o.getClass()) {
-      return false;
-    }
-    CacheDirectiveInfo other = (CacheDirectiveInfo)o;
-    return new EqualsBuilder().append(getId(), other.getId()).
-        append(getPath(), other.getPath()).
-        append(getReplication(), other.getReplication()).
-        append(getPool(), other.getPool()).
-        append(getExpiration(), other.getExpiration()).
-        isEquals();
-  }
-
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder().append(id).
-        append(path).
-        append(replication).
-        append(pool).
-        append(expiration).
-        hashCode();
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("{");
-    String prefix = "";
-    if (id != null) {
-      builder.append(prefix).append("id: ").append(id);
-      prefix = ", ";
-    }
-    if (path != null) {
-      builder.append(prefix).append("path: ").append(path);
-      prefix = ", ";
-    }
-    if (replication != null) {
-      builder.append(prefix).append("replication: ").append(replication);
-      prefix = ", ";
-    }
-    if (pool != null) {
-      builder.append(prefix).append("pool: ").append(pool);
-      prefix = ", ";
-    }
-    if (expiration != null) {
-      builder.append(prefix).append("expiration: ").append(expiration);
-      prefix = ", ";
-    }
-    builder.append("}");
-    return builder.toString();
-  }
-};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
deleted file mode 100644
index 0fd4ca2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
+++ /dev/null
@@ -1,169 +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;
-
-/**
- * Describes a path-based cache directive.
- */
-@InterfaceStability.Evolving
-@InterfaceAudience.Public
-public class CacheDirectiveStats {
-  public static class Builder {
-    private long bytesNeeded;
-    private long bytesCached;
-    private long filesNeeded;
-    private long filesCached;
-    private boolean hasExpired;
-
-    /**
-     * Builds a new CacheDirectiveStats populated with the set properties.
-     * 
-     * @return New CacheDirectiveStats.
-     */
-    public CacheDirectiveStats build() {
-      return new CacheDirectiveStats(bytesNeeded, bytesCached, filesNeeded,
-          filesCached, hasExpired);
-    }
-
-    /**
-     * Creates an empty builder.
-     */
-    public Builder() {
-    }
-
-    /**
-     * Sets the bytes needed by this directive.
-     * 
-     * @param bytesNeeded The bytes needed.
-     * @return This builder, for call chaining.
-     */
-    public Builder setBytesNeeded(long bytesNeeded) {
-      this.bytesNeeded = bytesNeeded;
-      return this;
-    }
-
-    /**
-     * Sets the bytes cached by this directive.
-     * 
-     * @param bytesCached The bytes cached.
-     * @return This builder, for call chaining.
-     */
-    public Builder setBytesCached(long bytesCached) {
-      this.bytesCached = bytesCached;
-      return this;
-    }
-
-    /**
-     * Sets the files needed by this directive.
-     * @param filesNeeded The number of files needed
-     * @return This builder, for call chaining.
-     */
-    public Builder setFilesNeeded(long filesNeeded) {
-      this.filesNeeded = filesNeeded;
-      return this;
-    }
-
-    /**
-     * Sets the files cached by this directive.
-     * 
-     * @param filesCached The number of files cached.
-     * @return This builder, for call chaining.
-     */
-    public Builder setFilesCached(long filesCached) {
-      this.filesCached = filesCached;
-      return this;
-    }
-
-    /**
-     * Sets whether this directive has expired.
-     * 
-     * @param hasExpired if this directive has expired
-     * @return This builder, for call chaining.
-     */
-    public Builder setHasExpired(boolean hasExpired) {
-      this.hasExpired = hasExpired;
-      return this;
-    }
-  }
-
-  private final long bytesNeeded;
-  private final long bytesCached;
-  private final long filesNeeded;
-  private final long filesCached;
-  private final boolean hasExpired;
-
-  private CacheDirectiveStats(long bytesNeeded, long bytesCached,
-      long filesNeeded, long filesCached, boolean hasExpired) {
-    this.bytesNeeded = bytesNeeded;
-    this.bytesCached = bytesCached;
-    this.filesNeeded = filesNeeded;
-    this.filesCached = filesCached;
-    this.hasExpired = hasExpired;
-  }
-
-  /**
-   * @return The bytes needed.
-   */
-  public long getBytesNeeded() {
-    return bytesNeeded;
-  }
-
-  /**
-   * @return The bytes cached.
-   */
-  public long getBytesCached() {
-    return bytesCached;
-  }
-
-  /**
-   * @return The number of files needed.
-   */
-  public long getFilesNeeded() {
-    return filesNeeded;
-  }
-
-  /**
-   * @return The number of files cached.
-   */
-  public long getFilesCached() {
-    return filesCached;
-  }
-
-  /**
-   * @return Whether this directive has expired.
-   */
-  public boolean hasExpired() {
-    return hasExpired;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("{");
-    builder.append("bytesNeeded: ").append(bytesNeeded);
-    builder.append(", ").append("bytesCached: ").append(bytesCached);
-    builder.append(", ").append("filesNeeded: ").append(filesNeeded);
-    builder.append(", ").append("filesCached: ").append(filesCached);
-    builder.append(", ").append("hasExpired: ").append(hasExpired);
-    builder.append("}");
-    return builder.toString();
-  }
-};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
deleted file mode 100644
index 3c1e345..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
+++ /dev/null
@@ -1,45 +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;
-
-/**
- * Describes a Cache Pool entry.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CachePoolEntry {
-  private final CachePoolInfo info;
-  private final CachePoolStats stats;
-
-  public CachePoolEntry(CachePoolInfo info, CachePoolStats stats) {
-    this.info = info;
-    this.stats = stats;
-  }
-
-  public CachePoolInfo getInfo() {
-    return info;
-  }
-
-  public CachePoolStats getStats() {
-    return stats;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
deleted file mode 100644
index 61bbe38..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
+++ /dev/null
@@ -1,229 +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 javax.annotation.Nullable;
-
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.InvalidRequestException;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
-
-/**
- * CachePoolInfo describes a cache pool.
- *
- * This class is used in RPCs to create and modify cache pools.
- * It is serializable and can be stored in the edit log.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CachePoolInfo {
-  public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
-
-  /**
-   * Indicates that the pool does not have a maximum relative expiry.
-   */
-  public static final long RELATIVE_EXPIRY_NEVER =
-      Expiration.MAX_RELATIVE_EXPIRY_MS;
-  /**
-   * Default max relative expiry for cache pools.
-   */
-  public static final long DEFAULT_MAX_RELATIVE_EXPIRY =
-      RELATIVE_EXPIRY_NEVER;
-
-  public static final long LIMIT_UNLIMITED = Long.MAX_VALUE;
-  public static final long DEFAULT_LIMIT = LIMIT_UNLIMITED;
-
-  final String poolName;
-
-  @Nullable
-  String ownerName;
-
-  @Nullable
-  String groupName;
-
-  @Nullable
-  FsPermission mode;
-
-  @Nullable
-  Long limit;
-
-  @Nullable
-  Long maxRelativeExpiryMs;
-
-  public CachePoolInfo(String poolName) {
-    this.poolName = poolName;
-  }
-
-  /**
-   * @return Name of the pool.
-   */
-  public String getPoolName() {
-    return poolName;
-  }
-
-  /**
-   * @return The owner of the pool. Along with the group and mode, determines
-   *         who has access to view and modify the pool.
-   */
-  public String getOwnerName() {
-    return ownerName;
-  }
-
-  public CachePoolInfo setOwnerName(String ownerName) {
-    this.ownerName = ownerName;
-    return this;
-  }
-
-  /**
-   * @return The group of the pool. Along with the owner and mode, determines
-   *         who has access to view and modify the pool.
-   */
-  public String getGroupName() {
-    return groupName;
-  }
-
-  public CachePoolInfo setGroupName(String groupName) {
-    this.groupName = groupName;
-    return this;
-  }
-
-  /**
-   * @return Unix-style permissions of the pool. Along with the owner and group,
-   *         determines who has access to view and modify the pool.
-   */
-  public FsPermission getMode() {
-    return mode;
-  }
-
-  public CachePoolInfo setMode(FsPermission mode) {
-    this.mode = mode;
-    return this;
-  }
-
-  /**
-   * @return The maximum aggregate number of bytes that can be cached by
-   *         directives in this pool.
-   */
-  public Long getLimit() {
-    return limit;
-  }
-
-  public CachePoolInfo setLimit(Long bytes) {
-    this.limit = bytes;
-    return this;
-  }
-
-  /**
-   * @return The maximum relative expiration of directives of this pool in
-   *         milliseconds
-   */
-  public Long getMaxRelativeExpiryMs() {
-    return maxRelativeExpiryMs;
-  }
-
-  /**
-   * Set the maximum relative expiration of directives of this pool in
-   * milliseconds.
-   * 
-   * @param ms in milliseconds
-   * @return This builder, for call chaining.
-   */
-  public CachePoolInfo setMaxRelativeExpiryMs(Long ms) {
-    this.maxRelativeExpiryMs = ms;
-    return this;
-  }
-
-  public String toString() {
-    return new StringBuilder().append("{").
-      append("poolName:").append(poolName).
-      append(", ownerName:").append(ownerName).
-      append(", groupName:").append(groupName).
-      append(", mode:").append((mode == null) ? "null" :
-          String.format("0%03o", mode.toShort())).
-      append(", limit:").append(limit).
-      append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
-      append("}").toString();
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (o == null) { return false; }
-    if (o == this) { return true; }
-    if (o.getClass() != getClass()) {
-      return false;
-    }
-    CachePoolInfo other = (CachePoolInfo)o;
-    return new EqualsBuilder().
-        append(poolName, other.poolName).
-        append(ownerName, other.ownerName).
-        append(groupName, other.groupName).
-        append(mode, other.mode).
-        append(limit, other.limit).
-        append(maxRelativeExpiryMs, other.maxRelativeExpiryMs).
-        isEquals();
-  }
-
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder().
-        append(poolName).
-        append(ownerName).
-        append(groupName).
-        append(mode).
-        append(limit).
-        append(maxRelativeExpiryMs).
-        hashCode();
-  }
-
-  public static void validate(CachePoolInfo info) throws IOException {
-    if (info == null) {
-      throw new InvalidRequestException("CachePoolInfo is null");
-    }
-    if ((info.getLimit() != null) && (info.getLimit() < 0)) {
-      throw new InvalidRequestException("Limit is negative.");
-    }
-    if (info.getMaxRelativeExpiryMs() != null) {
-      long maxRelativeExpiryMs = info.getMaxRelativeExpiryMs();
-      if (maxRelativeExpiryMs < 0l) {
-        throw new InvalidRequestException("Max relative expiry is negative.");
-      }
-      if (maxRelativeExpiryMs > Expiration.MAX_RELATIVE_EXPIRY_MS) {
-        throw new InvalidRequestException("Max relative expiry is too big.");
-      }
-    }
-    validateName(info.poolName);
-  }
-
-  public static void validateName(String poolName) throws IOException {
-    if (poolName == null || poolName.isEmpty()) {
-      // Empty pool names are not allowed because they would be highly
-      // confusing.  They would also break the ability to list all pools
-      // by starting with prevKey = ""
-      throw new IOException("invalid empty cache pool name");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
deleted file mode 100644
index c552652..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
+++ /dev/null
@@ -1,115 +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;
-
-/**
- * CachePoolStats describes cache pool statistics.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CachePoolStats {
-  public static class Builder {
-    private long bytesNeeded;
-    private long bytesCached;
-    private long bytesOverlimit;
-    private long filesNeeded;
-    private long filesCached;
-
-    public Builder() {
-    }
-
-    public Builder setBytesNeeded(long bytesNeeded) {
-      this.bytesNeeded = bytesNeeded;
-      return this;
-    }
-
-    public Builder setBytesCached(long bytesCached) {
-      this.bytesCached = bytesCached;
-      return this;
-    }
-
-    public Builder setBytesOverlimit(long bytesOverlimit) {
-      this.bytesOverlimit = bytesOverlimit;
-      return this;
-    }
-
-    public Builder setFilesNeeded(long filesNeeded) {
-      this.filesNeeded = filesNeeded;
-      return this;
-    }
-
-    public Builder setFilesCached(long filesCached) {
-      this.filesCached = filesCached;
-      return this;
-    }
-
-    public CachePoolStats build() {
-      return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
-          filesNeeded, filesCached);
-    }
-  };
-
-  private final long bytesNeeded;
-  private final long bytesCached;
-  private final long bytesOverlimit;
-  private final long filesNeeded;
-  private final long filesCached;
-
-  private CachePoolStats(long bytesNeeded, long bytesCached,
-      long bytesOverlimit, long filesNeeded, long filesCached) {
-    this.bytesNeeded = bytesNeeded;
-    this.bytesCached = bytesCached;
-    this.bytesOverlimit = bytesOverlimit;
-    this.filesNeeded = filesNeeded;
-    this.filesCached = filesCached;
-  }
-
-  public long getBytesNeeded() {
-    return bytesNeeded;
-  }
-
-  public long getBytesCached() {
-    return bytesCached;
-  }
-
-  public long getBytesOverlimit() {
-    return bytesOverlimit;
-  }
-
-  public long getFilesNeeded() {
-    return filesNeeded;
-  }
-
-  public long getFilesCached() {
-    return filesCached;
-  }
-
-  public String toString() {
-    return new StringBuilder().append("{").
-      append("bytesNeeded:").append(bytesNeeded).
-      append(", bytesCached:").append(bytesCached).
-      append(", bytesOverlimit:").append(bytesOverlimit).
-      append(", filesNeeded:").append(filesNeeded).
-      append(", filesCached:").append(filesCached).
-      append("}").toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
deleted file mode 100644
index b0db838..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
+++ /dev/null
@@ -1,215 +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.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtil;
-
-import com.google.common.base.Objects;
-
-/**
- * This class represents to end users the difference between two snapshots of 
- * the same directory, or the difference between a snapshot of the directory and
- * its current state. Instead of capturing all the details of the diff, this
- * class only lists where the changes happened and their types.
- */
-public class SnapshotDiffReport {
-  private final static String LINE_SEPARATOR = System.getProperty(
-      "line.separator", "\n");
-
-  /**
-   * Types of the difference, which include CREATE, MODIFY, DELETE, and RENAME.
-   * Each type has a label for representation: +/M/-/R represent CREATE, MODIFY,
-   * DELETE, and RENAME respectively.
-   */
-  public enum DiffType {
-    CREATE("+"),     
-    MODIFY("M"),    
-    DELETE("-"), 
-    RENAME("R");
-    
-    private final String label;
-    
-    private DiffType(String label) {
-      this.label = label;
-    }
-    
-    public String getLabel() {
-      return label;
-    }
-    
-    public static DiffType getTypeFromLabel(String label) {
-      if (label.equals(CREATE.getLabel())) {
-        return CREATE;
-      } else if (label.equals(MODIFY.getLabel())) {
-        return MODIFY;
-      } else if (label.equals(DELETE.getLabel())) {
-        return DELETE;
-      } else if (label.equals(RENAME.getLabel())) {
-        return RENAME;
-      }
-      return null;
-    }
-  };
-  
-  /**
-   * Representing the full path and diff type of a file/directory where changes
-   * have happened.
-   */
-  public static class DiffReportEntry {
-    /** The type of the difference. */
-    private final DiffType type;
-    /**
-     * The relative path (related to the snapshot root) of 1) the file/directory
-     * where changes have happened, or 2) the source file/dir of a rename op.
-     */
-    private final byte[] sourcePath;
-    private final byte[] targetPath;
-
-    public DiffReportEntry(DiffType type, byte[] sourcePath) {
-      this(type, sourcePath, null);
-    }
-
-    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents) {
-      this(type, sourcePathComponents, null);
-    }
-
-    public DiffReportEntry(DiffType type, byte[] sourcePath, byte[] targetPath) {
-      this.type = type;
-      this.sourcePath = sourcePath;
-      this.targetPath = targetPath;
-    }
-    
-    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
-        byte[][] targetPathComponents) {
-      this.type = type;
-      this.sourcePath = DFSUtil.byteArray2bytes(sourcePathComponents);
-      this.targetPath = targetPathComponents == null ? null : DFSUtil
-          .byteArray2bytes(targetPathComponents);
-    }
-    
-    @Override
-    public String toString() {
-      String str = type.getLabel() + "\t" + getPathString(sourcePath);
-      if (type == DiffType.RENAME) {
-        str += " -> " + getPathString(targetPath);
-      }
-      return str;
-    }
-    
-    public DiffType getType() {
-      return type;
-    }
-
-    static String getPathString(byte[] path) {
-      String pathStr = DFSUtil.bytes2String(path);
-      if (pathStr.isEmpty()) {
-        return Path.CUR_DIR;
-      } else {
-        return Path.CUR_DIR + Path.SEPARATOR + pathStr;
-      }
-    }
-
-    public byte[] getSourcePath() {
-      return sourcePath;
-    }
-
-    public byte[] getTargetPath() {
-      return targetPath;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (this == other) {
-        return true;
-      } 
-      if (other != null && other instanceof DiffReportEntry) {
-        DiffReportEntry entry = (DiffReportEntry) other;
-        return type.equals(entry.getType())
-            && Arrays.equals(sourcePath, entry.getSourcePath())
-            && Arrays.equals(targetPath, entry.getTargetPath());
-      }
-      return false;
-    }
-    
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(getSourcePath(), getTargetPath());
-    }
-  }
-  
-  /** snapshot root full path */
-  private final String snapshotRoot;
-
-  /** start point of the diff */
-  private final String fromSnapshot;
-  
-  /** end point of the diff */
-  private final String toSnapshot;
-  
-  /** list of diff */
-  private final List<DiffReportEntry> diffList;
-  
-  public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
-      String toSnapshot, List<DiffReportEntry> entryList) {
-    this.snapshotRoot = snapshotRoot;
-    this.fromSnapshot = fromSnapshot;
-    this.toSnapshot = toSnapshot;
-    this.diffList = entryList != null ? entryList : Collections
-        .<DiffReportEntry> emptyList();
-  }
-  
-  /** @return {@link #snapshotRoot}*/
-  public String getSnapshotRoot() {
-    return snapshotRoot;
-  }
-
-  /** @return {@link #fromSnapshot} */
-  public String getFromSnapshot() {
-    return fromSnapshot;
-  }
-
-  /** @return {@link #toSnapshot} */
-  public String getLaterSnapshotName() {
-    return toSnapshot;
-  }
-  
-  /** @return {@link #diffList} */
-  public List<DiffReportEntry> getDiffList() {
-    return diffList;
-  }
-  
-  @Override
-  public String toString() {
-    StringBuilder str = new StringBuilder();
-    String from = fromSnapshot == null || fromSnapshot.isEmpty() ? 
-        "current directory" : "snapshot " + fromSnapshot;
-    String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
-        : "snapshot " + toSnapshot;
-    str.append("Difference between " + from + " and " + to
-        + " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
-    for (DiffReportEntry entry : diffList) {
-      str.append(entry.toString() + LINE_SEPARATOR);
-    }
-    return str.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
deleted file mode 100644
index 3067696..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
+++ /dev/null
@@ -1,218 +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.PrintStream;
-import java.text.SimpleDateFormat;
-import java.util.Comparator;
-import java.util.Date;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSUtil;
-
-/**
- * Metadata about a snapshottable directory
- */
-public class SnapshottableDirectoryStatus {
-  /** Compare the statuses by full paths. */
-  public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
-      = new Comparator<SnapshottableDirectoryStatus>() {
-    @Override
-    public int compare(SnapshottableDirectoryStatus left,
-                       SnapshottableDirectoryStatus right) {
-      int d = DFSUtil.compareBytes(left.parentFullPath, right.parentFullPath);
-      return d != 0? d
-          : DFSUtil.compareBytes(left.dirStatus.getLocalNameInBytes(),
-              right.dirStatus.getLocalNameInBytes());
-    }
-  };
-
-  /** Basic information of the snapshottable directory */
-  private final HdfsFileStatus dirStatus;
-  
-  /** Number of snapshots that have been taken*/
-  private final int snapshotNumber;
-  
-  /** Number of snapshots allowed. */
-  private final int snapshotQuota;
-  
-  /** Full path of the parent. */
-  private final byte[] parentFullPath;
-  
-  public SnapshottableDirectoryStatus(long modification_time, long access_time,
-      FsPermission permission, String owner, String group, byte[] localName,
-      long inodeId, int childrenNum,
-      int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
-    this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
-        access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
-    this.snapshotNumber = snapshotNumber;
-    this.snapshotQuota = snapshotQuota;
-    this.parentFullPath = parentFullPath;
-  }
-
-  /**
-   * @return Number of snapshots that have been taken for the directory
-   */
-  public int getSnapshotNumber() {
-    return snapshotNumber;
-  }
-
-  /**
-   * @return Number of snapshots allowed for the directory
-   */
-  public int getSnapshotQuota() {
-    return snapshotQuota;
-  }
-  
-  /**
-   * @return Full path of the parent
-   */
-  public byte[] getParentFullPath() {
-    return parentFullPath;
-  }
-
-  /**
-   * @return The basic information of the directory
-   */
-  public HdfsFileStatus getDirStatus() {
-    return dirStatus;
-  }
-  
-  /**
-   * @return Full path of the file
-   */
-  public Path getFullPath() {
-    String parentFullPathStr = 
-        (parentFullPath == null || parentFullPath.length == 0) ? 
-            null : DFSUtil.bytes2String(parentFullPath);
-    if (parentFullPathStr == null
-        && dirStatus.getLocalNameInBytes().length == 0) {
-      // root
-      return new Path("/");
-    } else {
-      return parentFullPathStr == null ? new Path(dirStatus.getLocalName())
-          : new Path(parentFullPathStr, dirStatus.getLocalName());
-    }
-  }
-  
-  /**
-   * Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
-   * @param stats The list of {@link SnapshottableDirectoryStatus}
-   * @param out The given stream for printing.
-   */
-  public static void print(SnapshottableDirectoryStatus[] stats, 
-      PrintStream out) {
-    if (stats == null || stats.length == 0) {
-      out.println();
-      return;
-    }
-    int maxRepl = 0, maxLen = 0, maxOwner = 0, maxGroup = 0;
-    int maxSnapshotNum = 0, maxSnapshotQuota = 0;
-    for (SnapshottableDirectoryStatus status : stats) {
-      maxRepl = maxLength(maxRepl, status.dirStatus.getReplication());
-      maxLen = maxLength(maxLen, status.dirStatus.getLen());
-      maxOwner = maxLength(maxOwner, status.dirStatus.getOwner());
-      maxGroup = maxLength(maxGroup, status.dirStatus.getGroup());
-      maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
-      maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
-    }
-    
-    StringBuilder fmt = new StringBuilder();
-    fmt.append("%s%s "); // permission string
-    fmt.append("%"  + maxRepl  + "s ");
-    fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
-    fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
-    fmt.append("%"  + maxLen   + "s ");
-    fmt.append("%s "); // mod time
-    fmt.append("%"  + maxSnapshotNum  + "s ");
-    fmt.append("%"  + maxSnapshotQuota  + "s ");
-    fmt.append("%s"); // path
-    
-    String lineFormat = fmt.toString();
-    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
-         
-    for (SnapshottableDirectoryStatus status : stats) {
-      String line = String.format(lineFormat, "d", 
-          status.dirStatus.getPermission(),
-          status.dirStatus.getReplication(),
-          status.dirStatus.getOwner(),
-          status.dirStatus.getGroup(),
-          String.valueOf(status.dirStatus.getLen()),
-          dateFormat.format(new Date(status.dirStatus.getModificationTime())),
-          status.snapshotNumber, status.snapshotQuota, 
-          status.getFullPath().toString()
-      );
-      out.println(line);
-    }
-  }
-
-  private static int maxLength(int n, Object value) {
-    return Math.max(n, String.valueOf(value).length());
-  }
-
-  public static class Bean {
-    private final String path;
-    private final int snapshotNumber;
-    private final int snapshotQuota;
-    private final long modificationTime;
-    private final short permission;
-    private final String owner;
-    private final String group;
-
-    public Bean(String path, int snapshotNumber, int snapshotQuota,
-        long modificationTime, short permission, String owner, String group) {
-      this.path = path;
-      this.snapshotNumber = snapshotNumber;
-      this.snapshotQuota = snapshotQuota;
-      this.modificationTime = modificationTime;
-      this.permission = permission;
-      this.owner = owner;
-      this.group = group;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public int getSnapshotNumber() {
-      return snapshotNumber;
-    }
-
-    public int getSnapshotQuota() {
-      return snapshotQuota;
-    }
-
-    public long getModificationTime() {
-      return modificationTime;
-    }
-
-    public short getPermission() {
-      return permission;
-    }
-
-    public String getOwner() {
-      return owner;
-    }
-
-    public String getGroup() {
-      return group;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git 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
index ba6670c..c9a9c33 100644
--- 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,9 +47,9 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.inotify.Event;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -469,7 +469,7 @@ public class PBHelper {
   public static BlockKeyProto convert(BlockKey key) {
     byte[] encodedKey = key.getEncodedKey();
     ByteString keyBytes = ByteString.copyFrom(encodedKey == null ? 
-        DFSUtil.EMPTY_BYTES : encodedKey);
+        DFSUtilClient.EMPTY_BYTES : encodedKey);
     return BlockKeyProto.newBuilder().setKeyId(key.getKeyId())
         .setKeyBytes(keyBytes).setExpiryDate(key.getExpiryDate()).build();
   }
@@ -1514,7 +1514,7 @@ public class PBHelper {
     int snapshotQuota = status.getSnapshotQuota();
     byte[] parentFullPath = status.getParentFullPath();
     ByteString parentFullPathBytes = ByteString.copyFrom(
-        parentFullPath == null ? DFSUtil.EMPTY_BYTES : parentFullPath);
+        parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
     HdfsFileStatusProto fs = convert(status.getDirStatus());
     SnapshottableDirectoryStatusProto.Builder builder = 
         SnapshottableDirectoryStatusProto
@@ -2024,7 +2024,7 @@ public class PBHelper {
       return null;
     }
     ByteString sourcePath = ByteString
-        .copyFrom(entry.getSourcePath() == null ? DFSUtil.EMPTY_BYTES : entry
+        .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
             .getSourcePath());
     String modification = entry.getType().getLabel();
     SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
@@ -2032,7 +2032,7 @@ public class PBHelper {
         .setModificationLabel(modification);
     if (entry.getType() == DiffType.RENAME) {
       ByteString targetPath = ByteString
-          .copyFrom(entry.getTargetPath() == null ? DFSUtil.EMPTY_BYTES : entry
+          .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
               .getTargetPath());
       builder.setTargetPath(targetPath);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
deleted file mode 100644
index e9b24ca..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
+++ /dev/null
@@ -1,74 +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.security.token.delegation;
-
-import java.net.URI;
-import java.util.Collection;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
-
-/**
- * A delegation token that is specialized for HDFS
- */
-@InterfaceAudience.Private
-public class DelegationTokenSelector
-    extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
-  public static final String SERVICE_NAME_KEY = "hdfs.service.host_";
-
-  /**
-   * Select the delegation token for hdfs.  The port will be rewritten to
-   * the port of hdfs.service.host_$nnAddr, or the default rpc namenode port. 
-   * This method should only be called by non-hdfs filesystems that do not
-   * use the rpc port to acquire tokens.  Ex. webhdfs
-   * @param nnUri of the remote namenode
-   * @param tokens as a collection
-   * @param conf hadoop configuration
-   * @return Token
-   */
-  public Token<DelegationTokenIdentifier> selectToken(
-      final URI nnUri, Collection<Token<?>> tokens,
-      final Configuration conf) {
-    // this guesses the remote cluster's rpc service port.
-    // the current token design assumes it's the same as the local cluster's
-    // rpc port unless a config key is set.  there should be a way to automatic
-    // and correctly determine the value
-    Text serviceName = SecurityUtil.buildTokenService(nnUri);
-    final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
-    
-    int nnRpcPort = NameNode.DEFAULT_PORT;
-    if (nnServiceName != null) {
-      nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
-    }
-    // use original hostname from the uri to avoid unintentional host resolving
-    serviceName = SecurityUtil.buildTokenService(
-    		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
-    
-    return selectToken(serviceName, tokens);
-  }
-
-  public DelegationTokenSelector() {
-    super(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 409967e..f8efd76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -765,7 +766,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   @Override
   public final int compareTo(byte[] bytes) {
-    return DFSUtil.compareBytes(getLocalNameBytes(), bytes);
+    return DFSUtilClient.compareBytes(getLocalNameBytes(), bytes);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
deleted file mode 100644
index 712676d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
+++ /dev/null
@@ -1,37 +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;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * The file has not finished being written to enough datanodes yet.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class NotReplicatedYetException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public NotReplicatedYetException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index c3b7523..802d64a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import javax.management.ObjectName;
 
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
@@ -342,7 +343,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
             dir.getChildrenNum(Snapshot.CURRENT_STATE_ID),
             dir.getDirectorySnapshottableFeature().getNumSnapshots(),
             dir.getDirectorySnapshottableFeature().getSnapshotQuota(),
-            dir.getParent() == null ? DFSUtil.EMPTY_BYTES :
+            dir.getParent() == null ? DFSUtilClient.EMPTY_BYTES :
                 DFSUtil.string2Bytes(dir.getParent().getFullPathName()));
         statusList.add(status);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
deleted file mode 100644
index 0c8b6c93..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
+++ /dev/null
@@ -1,126 +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.protocol;
-
-import org.apache.hadoop.fs.StorageType;
-
-import java.util.UUID;
-
-/**
- * Class captures information of a storage in Datanode.
- */
-public class DatanodeStorage {
-  /** The state of the storage. */
-  public enum State {
-    NORMAL,
-    
-    /**
-     * A storage that represents a read-only path to replicas stored on a shared storage device.
-     * Replicas on {@link #READ_ONLY_SHARED} storage are not counted towards live replicas.
-     * 
-     * <p>
-     * In certain implementations, a {@link #READ_ONLY_SHARED} storage may be correlated to 
-     * its {@link #NORMAL} counterpart using the {@link DatanodeStorage#storageID}.  This
-     * property should be used for debugging purposes only.
-     * </p> 
-     */
-    READ_ONLY_SHARED,
-
-    FAILED;
-  }
-  
-  private final String storageID;
-  private final State state;
-  private final StorageType storageType;
-  private static final String STORAGE_ID_PREFIX = "DS-";
-
-  /**
-   * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
-   */
-  public DatanodeStorage(String storageID) {
-    this(storageID, State.NORMAL, StorageType.DEFAULT);
-  }
-
-  public DatanodeStorage(String sid, State s, StorageType sm) {
-    this.storageID = sid;
-    this.state = s;
-    this.storageType = sm;
-  }
-
-  public String getStorageID() {
-    return storageID;
-  }
-
-  public State getState() {
-    return state;
-  }
-
-  public StorageType getStorageType() {
-    return storageType;
-  }
-
-  /**
-   * Generate new storage ID. The format of this string can be changed
-   * in the future without requiring that old storage IDs be updated.
-   *
-   * @return unique storage ID
-   */
-  public static String generateUuid() {
-    return STORAGE_ID_PREFIX + UUID.randomUUID();
-  }
-
-  /**
-   * Verify that a given string is a storage ID in the "DS-..uuid.." format.
-   */
-  public static boolean isValidStorageId(final String storageID) {
-    try {
-      // Attempt to parse the UUID.
-      if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
-        UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
-        return true;
-      }
-    } catch (IllegalArgumentException iae) {
-    }
-
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return "DatanodeStorage["+ storageID + "," + storageType + "," + state +"]";
-  }
-  
-  @Override
-  public boolean equals(Object other){
-    if (other == this) {
-      return true;
-    }
-
-    if ((other == null) ||
-        !(other instanceof DatanodeStorage)) {
-      return false;
-    }
-    DatanodeStorage otherStorage = (DatanodeStorage) other;
-    return otherStorage.getStorageID().compareTo(getStorageID()) == 0;
-  }
-
-  @Override
-  public int hashCode() {
-    return getStorageID().hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
deleted file mode 100644
index 6a956a0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
+++ /dev/null
@@ -1,42 +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.protocol;
-
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-
-/**
- * Class captures information of a datanode and its storages.
- */
-public class DatanodeStorageReport {
-  final DatanodeInfo datanodeInfo;
-  final StorageReport[] storageReports;
-
-  public DatanodeStorageReport(DatanodeInfo datanodeInfo,
-      StorageReport[] storageReports) {
-    this.datanodeInfo = datanodeInfo;
-    this.storageReports = storageReports;
-  }
-
-  public DatanodeInfo getDatanodeInfo() {
-    return datanodeInfo;
-  }
-
-  public StorageReport[] getStorageReports() {
-    return storageReports;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
deleted file mode 100644
index 5fd5733..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
+++ /dev/null
@@ -1,66 +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.protocol;
-
-/**
- * Utilization report for a Datanode storage
- */
-public class StorageReport {
-  private final DatanodeStorage storage;
-  private final boolean failed;
-  private final long capacity;
-  private final long dfsUsed;
-  private final long remaining;
-  private final long blockPoolUsed;
-
-  public static final StorageReport[] EMPTY_ARRAY = {};
-  
-  public StorageReport(DatanodeStorage storage, boolean failed,
-      long capacity, long dfsUsed, long remaining, long bpUsed) {
-    this.storage = storage;
-    this.failed = failed;
-    this.capacity = capacity;
-    this.dfsUsed = dfsUsed;
-    this.remaining = remaining;
-    this.blockPoolUsed = bpUsed;
-  }
-
-  public DatanodeStorage getStorage() {
-    return storage;
-  }
-
-  public boolean isFailed() {
-    return failed;
-  }
-
-  public long getCapacity() {
-    return capacity;
-  }
-
-  public long getDfsUsed() {
-    return dfsUsed;
-  }
-
-  public long getRemaining() {
-    return remaining;
-  }
-
-  public long getBlockPoolUsed() {
-    return blockPoolUsed;
-  }
-}