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 wa...@apache.org on 2015/09/22 20:26:48 UTC

[14/32] hadoop git commit: HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to PBHelperClient. Contributed by Mingliang Liu.

HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to PBHelperClient. 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/06022b8f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/06022b8f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/06022b8f

Branch: refs/heads/YARN-1197
Commit: 06022b8fdc40e50eaac63758246353058e8cfa6d
Parents: 8e01b0d
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Sep 21 18:53:13 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Mon Sep 21 18:53:13 2015 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   83 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 1975 ++++++++++++++++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |    3 +
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   83 -
 .../hdfs/protocol/datatransfer/Receiver.java    |   33 +-
 ...tDatanodeProtocolServerSideTranslatorPB.java |    6 +-
 .../protocolPB/ClientNamenodeProtocolPB.java    |    4 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  128 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  122 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |   10 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   12 +-
 ...rDatanodeProtocolServerSideTranslatorPB.java |    2 +-
 .../protocolPB/JournalProtocolTranslatorPB.java |    2 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |    3 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 2038 +-----------------
 .../QJournalProtocolTranslatorPB.java           |    3 +-
 .../hdfs/server/namenode/CacheManager.java      |    4 +-
 .../server/namenode/EncryptionZoneManager.java  |    4 +-
 .../hdfs/server/namenode/FSDirXAttrOp.java      |    3 +-
 .../hdfs/server/namenode/FSDirectory.java       |    7 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |   20 +-
 .../server/namenode/FSImageFormatPBINode.java   |    7 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |    6 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   32 +-
 24 files changed, 2289 insertions(+), 2301 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
new file mode 100644
index 0000000..23e8f57
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@ -0,0 +1,83 @@
+/**
+ * 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.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/** 
+ * Interface that represents the over the wire information
+ * including block locations for a file.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HdfsLocatedFileStatus extends HdfsFileStatus {
+  private final LocatedBlocks locations;
+
+  /**
+   * Constructor
+   * 
+   * @param length size
+   * @param isdir if this is directory
+   * @param block_replication the file's replication factor
+   * @param blocksize the file's block size
+   * @param modification_time most recent modification time
+   * @param access_time most recent access time
+   * @param permission permission
+   * @param owner owner
+   * @param group group
+   * @param symlink symbolic link
+   * @param path local path name in java UTF8 format 
+   * @param fileId the file id
+   * @param locations block locations
+   * @param feInfo file encryption info
+   */
+  public HdfsLocatedFileStatus(long length, boolean isdir,
+      int block_replication, long blocksize, long modification_time,
+      long access_time, FsPermission permission, String owner, String group,
+      byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
+      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
+    super(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path, fileId,
+        childrenNum, feInfo, storagePolicy);
+    this.locations = locations;
+  }
+
+  public LocatedBlocks getBlockLocations() {
+    return locations;
+  }
+
+  public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
+      Path path) {
+    return new LocatedFileStatus(getLen(), isDir(), getReplication(),
+        getBlockSize(), getModificationTime(),
+        getAccessTime(),
+        getPermission(), getOwner(), getGroup(),
+        isSymlink() ? new Path(getSymlink()) : null,
+        (getFullPath(path)).makeQualified(
+            defaultUri, null), // fully-qualify path
+        DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 98de2e9..ae0a3f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -17,46 +17,173 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.primitives.Shorts;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.hadoop.crypto.CipherOption;
 import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+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.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+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.DFSUtilClient;
+import org.apache.hadoop.hdfs.inotify.Event;
+import org.apache.hadoop.hdfs.inotify.EventBatch;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.Block;
+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.CacheDirectiveStats;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolStats;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
+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.FsPermissionExtension;
+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.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 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.InotifyProtos;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto.XAttrNamespaceProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrSetFlagProto;
+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.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-
 /**
- * Utilities for converting protobuf classes to and from implementation classes
- * and other helper utilities to help in dealing with protobuf.
+ * Utilities for converting protobuf classes to and from hdfs-client side
+ * implementation classes and other helper utilities to help in dealing with
+ * protobuf.
  *
  * Note that when converting from an internal type to protobuf type, the
  * converter never return null for protobuf type. The check for internal type
  * being null must be done before calling the convert() method.
  */
 public class PBHelperClient {
+  private static final XAttr.NameSpace[] XATTR_NAMESPACE_VALUES =
+      XAttr.NameSpace.values();
+  private static final AclEntryType[] ACL_ENTRY_TYPE_VALUES =
+      AclEntryType.values();
+  private static final AclEntryScope[] ACL_ENTRY_SCOPE_VALUES =
+      AclEntryScope.values();
+  private static final FsAction[] FSACTION_VALUES =
+      FsAction.values();
+
   private PBHelperClient() {
     /** Hidden constructor */
   }
@@ -253,7 +380,7 @@ public class PBHelperClient {
     final List<StorageTypeProto> protos = new ArrayList<>(
       types.length);
     for (int i = startIdx; i < types.length; ++i) {
-      protos.add(PBHelperClient.convertStorageType(types[i]));
+      protos.add(convertStorageType(types[i]));
     }
     return protos;
   }
@@ -369,4 +496,1834 @@ public class PBHelperClient {
     }
     return null;
   }
+
+  public static LocatedBlock convert(LocatedBlockProto proto) {
+    if (proto == null) return null;
+    List<DatanodeInfoProto> locs = proto.getLocsList();
+    DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
+    for (int i = 0; i < locs.size(); i++) {
+      targets[i] = convert(locs.get(i));
+    }
+
+    final StorageType[] storageTypes = convertStorageTypes(
+        proto.getStorageTypesList(), locs.size());
+
+    final int storageIDsCount = proto.getStorageIDsCount();
+    final String[] storageIDs;
+    if (storageIDsCount == 0) {
+      storageIDs = null;
+    } else {
+      Preconditions.checkState(storageIDsCount == locs.size());
+      storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
+    }
+
+    // Set values from the isCached list, re-using references from loc
+    List<DatanodeInfo> cachedLocs = new ArrayList<>(locs.size());
+    List<Boolean> isCachedList = proto.getIsCachedList();
+    for (int i=0; i<isCachedList.size(); i++) {
+      if (isCachedList.get(i)) {
+        cachedLocs.add(targets[i]);
+      }
+    }
+
+    LocatedBlock lb = new LocatedBlock(convert(proto.getB()), targets,
+        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
+        cachedLocs.toArray(new DatanodeInfo[0]));
+    lb.setBlockToken(convert(proto.getBlockToken()));
+
+    return lb;
+  }
+
+  static public DatanodeInfo convert(DatanodeInfoProto di) {
+    if (di == null) return null;
+    return new DatanodeInfo(
+        convert(di.getId()),
+        di.hasLocation() ? di.getLocation() : null,
+        di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
+        di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
+        di.getLastUpdate(), di.getLastUpdateMonotonic(),
+        di.getXceiverCount(), convert(di.getAdminState()),
+        di.hasUpgradeDomain() ? di.getUpgradeDomain() : null);
+  }
+
+  public static StorageType[] convertStorageTypes(
+      List<StorageTypeProto> storageTypesList, int expectedSize) {
+    final StorageType[] storageTypes = new StorageType[expectedSize];
+    if (storageTypesList.size() != expectedSize) {
+     // missing storage types
+      Preconditions.checkState(storageTypesList.isEmpty());
+      Arrays.fill(storageTypes, StorageType.DEFAULT);
+    } else {
+      for (int i = 0; i < storageTypes.length; ++i) {
+        storageTypes[i] = convertStorageType(storageTypesList.get(i));
+      }
+    }
+    return storageTypes;
+  }
+
+  public static Token<BlockTokenIdentifier> convert(
+      TokenProto blockToken) {
+    return new Token<>(blockToken.getIdentifier()
+        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+        blockToken.getKind()), new Text(blockToken.getService()));
+  }
+
+  // DatanodeId
+  public static DatanodeID convert(DatanodeIDProto dn) {
+    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
+        dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
+        .getInfoSecurePort() : 0, dn.getIpcPort());
+  }
+
+  public static AdminStates convert(AdminState adminState) {
+    switch(adminState) {
+    case DECOMMISSION_INPROGRESS:
+      return AdminStates.DECOMMISSION_INPROGRESS;
+    case DECOMMISSIONED:
+      return AdminStates.DECOMMISSIONED;
+    case NORMAL:
+    default:
+      return AdminStates.NORMAL;
+    }
+  }
+
+  // LocatedBlocks
+  public static LocatedBlocks convert(LocatedBlocksProto lb) {
+    return new LocatedBlocks(
+        lb.getFileLength(), lb.getUnderConstruction(),
+        convertLocatedBlock(lb.getBlocksList()),
+        lb.hasLastBlock() ? convert(lb.getLastBlock()) : null,
+        lb.getIsLastBlockComplete(),
+        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
+            null);
+  }
+
+  public static BlockStoragePolicy[] convertStoragePolicies(
+      List<BlockStoragePolicyProto> policyProtos) {
+    if (policyProtos == null || policyProtos.size() == 0) {
+      return new BlockStoragePolicy[0];
+    }
+    BlockStoragePolicy[] policies = new BlockStoragePolicy[policyProtos.size()];
+    int i = 0;
+    for (BlockStoragePolicyProto proto : policyProtos) {
+      policies[i++] = convert(proto);
+    }
+    return policies;
+  }
+
+  public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
+    IOException {
+    final InotifyProtos.EventsListProto list = resp.getEventsList();
+    final long firstTxid = list.getFirstTxid();
+    final long lastTxid = list.getLastTxid();
+
+    List<EventBatch> batches = Lists.newArrayList();
+    if (list.getEventsList().size() > 0) {
+      throw new IOException("Can't handle old inotify server response.");
+    }
+    for (InotifyProtos.EventBatchProto bp : list.getBatchList()) {
+      long txid = bp.getTxid();
+      if ((txid != -1) && ((txid < firstTxid) || (txid > lastTxid))) {
+        throw new IOException("Error converting TxidResponseProto: got a " +
+            "transaction id " + txid + " that was outside the range of [" +
+            firstTxid + ", " + lastTxid + "].");
+      }
+      List<Event> events = Lists.newArrayList();
+      for (InotifyProtos.EventProto p : bp.getEventsList()) {
+        switch (p.getType()) {
+          case EVENT_CLOSE:
+            InotifyProtos.CloseEventProto close =
+                InotifyProtos.CloseEventProto.parseFrom(p.getContents());
+            events.add(new Event.CloseEvent(close.getPath(),
+                close.getFileSize(), close.getTimestamp()));
+            break;
+          case EVENT_CREATE:
+            InotifyProtos.CreateEventProto create =
+                InotifyProtos.CreateEventProto.parseFrom(p.getContents());
+            events.add(new Event.CreateEvent.Builder()
+                .iNodeType(createTypeConvert(create.getType()))
+                .path(create.getPath())
+                .ctime(create.getCtime())
+                .ownerName(create.getOwnerName())
+                .groupName(create.getGroupName())
+                .perms(convert(create.getPerms()))
+                .replication(create.getReplication())
+                .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
+                    create.getSymlinkTarget())
+                .defaultBlockSize(create.getDefaultBlockSize())
+                .overwrite(create.getOverwrite()).build());
+            break;
+          case EVENT_METADATA:
+            InotifyProtos.MetadataUpdateEventProto meta =
+                InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
+            events.add(new Event.MetadataUpdateEvent.Builder()
+                .path(meta.getPath())
+                .metadataType(metadataUpdateTypeConvert(meta.getType()))
+                .mtime(meta.getMtime())
+                .atime(meta.getAtime())
+                .replication(meta.getReplication())
+                .ownerName(
+                    meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
+                .groupName(
+                    meta.getGroupName().isEmpty() ? null : meta.getGroupName())
+                .perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
+                .acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
+                    meta.getAclsList()))
+                .xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
+                    meta.getXAttrsList()))
+                .xAttrsRemoved(meta.getXAttrsRemoved())
+                .build());
+            break;
+          case EVENT_RENAME:
+            InotifyProtos.RenameEventProto rename =
+                InotifyProtos.RenameEventProto.parseFrom(p.getContents());
+            events.add(new Event.RenameEvent.Builder()
+                  .srcPath(rename.getSrcPath())
+                  .dstPath(rename.getDestPath())
+                  .timestamp(rename.getTimestamp())
+                  .build());
+            break;
+          case EVENT_APPEND:
+            InotifyProtos.AppendEventProto append =
+                InotifyProtos.AppendEventProto.parseFrom(p.getContents());
+            events.add(new Event.AppendEvent.Builder().path(append.getPath())
+                .newBlock(append.hasNewBlock() && append.getNewBlock())
+                .build());
+            break;
+          case EVENT_UNLINK:
+            InotifyProtos.UnlinkEventProto unlink =
+                InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
+            events.add(new Event.UnlinkEvent.Builder()
+                  .path(unlink.getPath())
+                  .timestamp(unlink.getTimestamp())
+                  .build());
+            break;
+          case EVENT_TRUNCATE:
+            InotifyProtos.TruncateEventProto truncate =
+                InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
+            events.add(new Event.TruncateEvent(truncate.getPath(),
+                truncate.getFileSize(), truncate.getTimestamp()));
+            break;
+          default:
+            throw new RuntimeException("Unexpected inotify event type: " +
+                p.getType());
+        }
+      }
+      batches.add(new EventBatch(txid, events.toArray(new Event[0])));
+    }
+    return new EventBatchList(batches, resp.getEventsList().getFirstTxid(),
+        resp.getEventsList().getLastTxid(), resp.getEventsList().getSyncTxid());
+  }
+
+  // Located Block Arrays and Lists
+  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
+    if (lb == null) return null;
+    return convertLocatedBlock2(Arrays.asList(lb)).toArray(
+        new LocatedBlockProto[lb.length]);
+  }
+
+  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
+    if (lb == null) return null;
+    final int len = lb.size();
+    List<LocatedBlockProto> result = new ArrayList<>(len);
+    for (int i = 0; i < len; ++i) {
+      result.add(convert(lb.get(i)));
+    }
+    return result;
+  }
+
+  public static LocatedBlockProto convert(LocatedBlock b) {
+    if (b == null) return null;
+    Builder builder = LocatedBlockProto.newBuilder();
+    DatanodeInfo[] locs = b.getLocations();
+    List<DatanodeInfo> cachedLocs =
+        Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
+    for (int i = 0; i < locs.length; i++) {
+      DatanodeInfo loc = locs[i];
+      builder.addLocs(i, convert(loc));
+      boolean locIsCached = cachedLocs.contains(loc);
+      builder.addIsCached(locIsCached);
+      if (locIsCached) {
+        cachedLocs.remove(loc);
+      }
+    }
+    Preconditions.checkArgument(cachedLocs.size() == 0,
+        "Found additional cached replica locations that are not in the set of"
+        + " storage-backed locations!");
+
+    StorageType[] storageTypes = b.getStorageTypes();
+    if (storageTypes != null) {
+      for (int i = 0; i < storageTypes.length; ++i) {
+        builder.addStorageTypes(convertStorageType(storageTypes[i]));
+      }
+    }
+    final String[] storageIDs = b.getStorageIDs();
+    if (storageIDs != null) {
+      builder.addAllStorageIDs(Arrays.asList(storageIDs));
+    }
+
+    return builder.setB(convert(b.getBlock()))
+        .setBlockToken(convert(b.getBlockToken()))
+        .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
+  }
+
+  public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) {
+    List<StorageTypeProto> cList = proto.getCreationPolicy()
+        .getStorageTypesList();
+    StorageType[] creationTypes = convertStorageTypes(cList, cList.size());
+    List<StorageTypeProto> cfList = proto.hasCreationFallbackPolicy() ? proto
+        .getCreationFallbackPolicy().getStorageTypesList() : null;
+    StorageType[] creationFallbackTypes = cfList == null ? StorageType
+        .EMPTY_ARRAY : convertStorageTypes(cfList, cfList.size());
+    List<StorageTypeProto> rfList = proto.hasReplicationFallbackPolicy() ?
+        proto.getReplicationFallbackPolicy().getStorageTypesList() : null;
+    StorageType[] replicationFallbackTypes = rfList == null ? StorageType
+        .EMPTY_ARRAY : convertStorageTypes(rfList, rfList.size());
+    return new BlockStoragePolicy((byte) proto.getPolicyId(), proto.getName(),
+        creationTypes, creationFallbackTypes, replicationFallbackTypes);
+  }
+
+  public static FsActionProto convert(FsAction v) {
+    return FsActionProto.valueOf(v != null ? v.ordinal() : 0);
+  }
+
+  public static XAttrProto convertXAttrProto(XAttr a) {
+    XAttrProto.Builder builder = XAttrProto.newBuilder();
+    builder.setNamespace(convert(a.getNameSpace()));
+    if (a.getName() != null) {
+      builder.setName(a.getName());
+    }
+    if (a.getValue() != null) {
+      builder.setValue(getByteString(a.getValue()));
+    }
+    return builder.build();
+  }
+
+  public static List<XAttr> convert(ListXAttrsResponseProto a) {
+    final List<XAttrProto> xAttrs = a.getXAttrsList();
+    return convertXAttrs(xAttrs);
+  }
+
+  public static List<XAttr> convert(GetXAttrsResponseProto a) {
+    List<XAttrProto> xAttrs = a.getXAttrsList();
+    return convertXAttrs(xAttrs);
+  }
+
+  public static List<XAttr> convertXAttrs(List<XAttrProto> xAttrSpec) {
+    ArrayList<XAttr> xAttrs = Lists.newArrayListWithCapacity(xAttrSpec.size());
+    for (XAttrProto a : xAttrSpec) {
+      XAttr.Builder builder = new XAttr.Builder();
+      builder.setNameSpace(convert(a.getNamespace()));
+      if (a.hasName()) {
+        builder.setName(a.getName());
+      }
+      if (a.hasValue()) {
+        builder.setValue(a.getValue().toByteArray());
+      }
+      xAttrs.add(builder.build());
+    }
+    return xAttrs;
+  }
+
+  static XAttrNamespaceProto convert(XAttr.NameSpace v) {
+    return XAttrNamespaceProto.valueOf(v.ordinal());
+  }
+
+  static XAttr.NameSpace convert(XAttrNamespaceProto v) {
+    return castEnum(v, XATTR_NAMESPACE_VALUES);
+  }
+
+  static <T extends Enum<T>, U extends Enum<U>> U castEnum(T from, U[] to) {
+    return to[from.ordinal()];
+  }
+
+  static InotifyProtos.MetadataUpdateType metadataUpdateTypeConvert(
+      Event.MetadataUpdateEvent.MetadataType type) {
+    switch (type) {
+    case TIMES:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_TIMES;
+    case REPLICATION:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_REPLICATION;
+    case OWNER:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_OWNER;
+    case PERMS:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_PERMS;
+    case ACLS:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_ACLS;
+    case XATTRS:
+      return InotifyProtos.MetadataUpdateType.META_TYPE_XATTRS;
+    default:
+      return null;
+    }
+  }
+
+  private static Event.MetadataUpdateEvent.MetadataType metadataUpdateTypeConvert(
+      InotifyProtos.MetadataUpdateType type) {
+    switch (type) {
+    case META_TYPE_TIMES:
+      return Event.MetadataUpdateEvent.MetadataType.TIMES;
+    case META_TYPE_REPLICATION:
+      return Event.MetadataUpdateEvent.MetadataType.REPLICATION;
+    case META_TYPE_OWNER:
+      return Event.MetadataUpdateEvent.MetadataType.OWNER;
+    case META_TYPE_PERMS:
+      return Event.MetadataUpdateEvent.MetadataType.PERMS;
+    case META_TYPE_ACLS:
+      return Event.MetadataUpdateEvent.MetadataType.ACLS;
+    case META_TYPE_XATTRS:
+      return Event.MetadataUpdateEvent.MetadataType.XATTRS;
+    default:
+      return null;
+    }
+  }
+
+  static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
+                                                       type) {
+    switch (type) {
+    case DIRECTORY:
+      return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
+    case FILE:
+      return InotifyProtos.INodeType.I_TYPE_FILE;
+    case SYMLINK:
+      return InotifyProtos.INodeType.I_TYPE_SYMLINK;
+    default:
+      return null;
+    }
+  }
+
+  public static List<LocatedBlock> convertLocatedBlock(
+      List<LocatedBlockProto> lb) {
+    if (lb == null) return null;
+    final int len = lb.size();
+    List<LocatedBlock> result = new ArrayList<>(len);
+    for (int i = 0; i < len; ++i) {
+      result.add(convert(lb.get(i)));
+    }
+    return result;
+  }
+
+  public static List<AclEntry> convertAclEntry(List<AclEntryProto> aclSpec) {
+    ArrayList<AclEntry> r = Lists.newArrayListWithCapacity(aclSpec.size());
+    for (AclEntryProto e : aclSpec) {
+      AclEntry.Builder builder = new AclEntry.Builder();
+      builder.setType(convert(e.getType()));
+      builder.setScope(convert(e.getScope()));
+      builder.setPermission(convert(e.getPermissions()));
+      if (e.hasName()) {
+        builder.setName(e.getName());
+      }
+      r.add(builder.build());
+    }
+    return r;
+  }
+
+  static AclEntryScopeProto convert(AclEntryScope v) {
+    return AclEntryScopeProto.valueOf(v.ordinal());
+  }
+
+  private static AclEntryScope convert(AclEntryScopeProto v) {
+    return castEnum(v, ACL_ENTRY_SCOPE_VALUES);
+  }
+
+  static AclEntryTypeProto convert(AclEntryType e) {
+    return AclEntryTypeProto.valueOf(e.ordinal());
+  }
+
+  private static AclEntryType convert(AclEntryTypeProto v) {
+    return castEnum(v, ACL_ENTRY_TYPE_VALUES);
+  }
+
+  public static FsAction convert(FsActionProto v) {
+    return castEnum(v, FSACTION_VALUES);
+  }
+
+  public static FsPermission convert(FsPermissionProto p) {
+    return new FsPermissionExtension((short)p.getPerm());
+  }
+
+  private static Event.CreateEvent.INodeType createTypeConvert(
+      InotifyProtos.INodeType type) {
+    switch (type) {
+    case I_TYPE_DIRECTORY:
+      return Event.CreateEvent.INodeType.DIRECTORY;
+    case I_TYPE_FILE:
+      return Event.CreateEvent.INodeType.FILE;
+    case I_TYPE_SYMLINK:
+      return Event.CreateEvent.INodeType.SYMLINK;
+    default:
+      return null;
+    }
+  }
+
+  public static HdfsProtos.FileEncryptionInfoProto convert(
+      FileEncryptionInfo info) {
+    if (info == null) {
+      return null;
+    }
+    return HdfsProtos.FileEncryptionInfoProto.newBuilder()
+        .setSuite(convert(info.getCipherSuite()))
+        .setCryptoProtocolVersion(convert(info.getCryptoProtocolVersion()))
+        .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+        .setIv(getByteString(info.getIV()))
+        .setEzKeyVersionName(info.getEzKeyVersionName())
+        .setKeyName(info.getKeyName())
+        .build();
+  }
+
+  public static CryptoProtocolVersionProto convert(CryptoProtocolVersion
+      version) {
+    switch(version) {
+    case UNKNOWN:
+      return CryptoProtocolVersionProto.UNKNOWN_PROTOCOL_VERSION;
+    case ENCRYPTION_ZONES:
+      return CryptoProtocolVersionProto.ENCRYPTION_ZONES;
+    default:
+      return null;
+    }
+  }
+
+  public static FileEncryptionInfo convert(
+      HdfsProtos.FileEncryptionInfoProto proto) {
+    if (proto == null) {
+      return null;
+    }
+    CipherSuite suite = convert(proto.getSuite());
+    CryptoProtocolVersion version = convert(proto.getCryptoProtocolVersion());
+    byte[] key = proto.getKey().toByteArray();
+    byte[] iv = proto.getIv().toByteArray();
+    String ezKeyVersionName = proto.getEzKeyVersionName();
+    String keyName = proto.getKeyName();
+    return new FileEncryptionInfo(suite, version, key, iv, keyName,
+        ezKeyVersionName);
+  }
+
+  public static CryptoProtocolVersion convert(CryptoProtocolVersionProto
+      proto) {
+    switch(proto) {
+    case ENCRYPTION_ZONES:
+      return CryptoProtocolVersion.ENCRYPTION_ZONES;
+    default:
+      // Set to UNKNOWN and stash the unknown enum value
+      CryptoProtocolVersion version = CryptoProtocolVersion.UNKNOWN;
+      version.setUnknownValue(proto.getNumber());
+      return version;
+    }
+  }
+
+  public static List<XAttrProto> convertXAttrProto(
+      List<XAttr> xAttrSpec) {
+    if (xAttrSpec == null) {
+      return Lists.newArrayListWithCapacity(0);
+    }
+    ArrayList<XAttrProto> xAttrs = Lists.newArrayListWithCapacity(
+        xAttrSpec.size());
+    for (XAttr a : xAttrSpec) {
+      XAttrProto.Builder builder = XAttrProto.newBuilder();
+      builder.setNamespace(convert(a.getNameSpace()));
+      if (a.getName() != null) {
+        builder.setName(a.getName());
+      }
+      if (a.getValue() != null) {
+        builder.setValue(getByteString(a.getValue()));
+      }
+      xAttrs.add(builder.build());
+    }
+    return xAttrs;
+  }
+
+  /**
+   * The flag field in PB is a bitmask whose values are the same a the
+   * emum values of XAttrSetFlag
+   */
+  public static int convert(EnumSet<XAttrSetFlag> flag) {
+    int value = 0;
+    if (flag.contains(XAttrSetFlag.CREATE)) {
+      value |= XAttrSetFlagProto.XATTR_CREATE.getNumber();
+    }
+    if (flag.contains(XAttrSetFlag.REPLACE)) {
+      value |= XAttrSetFlagProto.XATTR_REPLACE.getNumber();
+    }
+    return value;
+  }
+
+  public static EncryptionZone convert(EncryptionZoneProto proto) {
+    return new EncryptionZone(proto.getId(), proto.getPath(),
+        convert(proto.getSuite()), convert(proto.getCryptoProtocolVersion()),
+        proto.getKeyName());
+  }
+
+  public static AclStatus convert(GetAclStatusResponseProto e) {
+    AclStatusProto r = e.getResult();
+    AclStatus.Builder builder = new AclStatus.Builder();
+    builder.owner(r.getOwner()).group(r.getGroup()).stickyBit(r.getSticky())
+        .addEntries(convertAclEntry(r.getEntriesList()));
+    if (r.hasPermission()) {
+      builder.setPermission(convert(r.getPermission()));
+    }
+    return builder.build();
+  }
+
+  public static List<AclEntryProto> convertAclEntryProto(
+      List<AclEntry> aclSpec) {
+    ArrayList<AclEntryProto> r = Lists.newArrayListWithCapacity(aclSpec.size());
+    for (AclEntry e : aclSpec) {
+      AclEntryProto.Builder builder = AclEntryProto.newBuilder();
+      builder.setType(convert(e.getType()));
+      builder.setScope(convert(e.getScope()));
+      builder.setPermissions(convert(e.getPermission()));
+      if (e.getName() != null) {
+        builder.setName(e.getName());
+      }
+      r.add(builder.build());
+    }
+    return r;
+  }
+
+  public static CachePoolEntry convert(CachePoolEntryProto proto) {
+    CachePoolInfo info = convert(proto.getInfo());
+    CachePoolStats stats = convert(proto.getStats());
+    return new CachePoolEntry(info, stats);
+  }
+
+  public static CachePoolInfo convert (CachePoolInfoProto proto) {
+    // Pool name is a required field, the rest are optional
+    String poolName = checkNotNull(proto.getPoolName());
+    CachePoolInfo info = new CachePoolInfo(poolName);
+    if (proto.hasOwnerName()) {
+        info.setOwnerName(proto.getOwnerName());
+    }
+    if (proto.hasGroupName()) {
+      info.setGroupName(proto.getGroupName());
+    }
+    if (proto.hasMode()) {
+      info.setMode(new FsPermission((short)proto.getMode()));
+    }
+    if (proto.hasLimit())  {
+      info.setLimit(proto.getLimit());
+    }
+    if (proto.hasMaxRelativeExpiry()) {
+      info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
+    }
+    return info;
+  }
+
+  public static CachePoolStats convert (CachePoolStatsProto proto) {
+    CachePoolStats.Builder builder = new CachePoolStats.Builder();
+    builder.setBytesNeeded(proto.getBytesNeeded());
+    builder.setBytesCached(proto.getBytesCached());
+    builder.setBytesOverlimit(proto.getBytesOverlimit());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
+    return builder.build();
+  }
+
+  public static CachePoolInfoProto convert(CachePoolInfo info) {
+    CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
+    builder.setPoolName(info.getPoolName());
+    if (info.getOwnerName() != null) {
+      builder.setOwnerName(info.getOwnerName());
+    }
+    if (info.getGroupName() != null) {
+      builder.setGroupName(info.getGroupName());
+    }
+    if (info.getMode() != null) {
+      builder.setMode(info.getMode().toShort());
+    }
+    if (info.getLimit() != null) {
+      builder.setLimit(info.getLimit());
+    }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfoProto convert
+      (CacheDirectiveInfo info) {
+    CacheDirectiveInfoProto.Builder builder =
+        CacheDirectiveInfoProto.newBuilder();
+    if (info.getId() != null) {
+      builder.setId(info.getId());
+    }
+    if (info.getPath() != null) {
+      builder.setPath(info.getPath().toUri().getPath());
+    }
+    if (info.getReplication() != null) {
+      builder.setReplication(info.getReplication());
+    }
+    if (info.getPool() != null) {
+      builder.setPool(info.getPool());
+    }
+    if (info.getExpiration() != null) {
+      builder.setExpiration(convert(info.getExpiration()));
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfoExpirationProto convert(
+      CacheDirectiveInfo.Expiration expiration) {
+    return CacheDirectiveInfoExpirationProto.newBuilder()
+        .setIsRelative(expiration.isRelative())
+        .setMillis(expiration.getMillis())
+        .build();
+  }
+
+  public static CacheDirectiveEntry convert(CacheDirectiveEntryProto proto) {
+    CacheDirectiveInfo info = convert(proto.getInfo());
+    CacheDirectiveStats stats = convert(proto.getStats());
+    return new CacheDirectiveEntry(info, stats);
+  }
+
+  public static CacheDirectiveStats convert(CacheDirectiveStatsProto proto) {
+    CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+    builder.setBytesNeeded(proto.getBytesNeeded());
+    builder.setBytesCached(proto.getBytesCached());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
+    builder.setHasExpired(proto.getHasExpired());
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfo convert
+      (CacheDirectiveInfoProto proto) {
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder();
+    if (proto.hasId()) {
+      builder.setId(proto.getId());
+    }
+    if (proto.hasPath()) {
+      builder.setPath(new Path(proto.getPath()));
+    }
+    if (proto.hasReplication()) {
+      builder.setReplication(Shorts.checkedCast(
+          proto.getReplication()));
+    }
+    if (proto.hasPool()) {
+      builder.setPool(proto.getPool());
+    }
+    if (proto.hasExpiration()) {
+      builder.setExpiration(convert(proto.getExpiration()));
+    }
+    return builder.build();
+  }
+
+  public static CacheDirectiveInfo.Expiration convert(
+      CacheDirectiveInfoExpirationProto proto) {
+    if (proto.getIsRelative()) {
+      return CacheDirectiveInfo.Expiration.newRelative(proto.getMillis());
+    }
+    return CacheDirectiveInfo.Expiration.newAbsolute(proto.getMillis());
+  }
+
+  public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
+    int value = 0;
+    if (flags.contains(CacheFlag.FORCE)) {
+      value |= CacheFlagProto.FORCE.getNumber();
+    }
+    return value;
+  }
+
+  public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
+    if (reportProto == null) {
+      return null;
+    }
+    String snapshotDir = reportProto.getSnapshotRoot();
+    String fromSnapshot = reportProto.getFromSnapshot();
+    String toSnapshot = reportProto.getToSnapshot();
+    List<SnapshotDiffReportEntryProto> list = reportProto
+        .getDiffReportEntriesList();
+    List<DiffReportEntry> entries = new ArrayList<>();
+    for (SnapshotDiffReportEntryProto entryProto : list) {
+      DiffReportEntry entry = convert(entryProto);
+      if (entry != null)
+        entries.add(entry);
+    }
+    return new SnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot,
+        entries);
+  }
+
+  public static DiffReportEntry convert(SnapshotDiffReportEntryProto entry) {
+    if (entry == null) {
+      return null;
+    }
+    DiffType type = DiffType.getTypeFromLabel(entry
+        .getModificationLabel());
+    return type == null ? null : new DiffReportEntry(type, entry.getFullpath()
+        .toByteArray(), entry.hasTargetPath() ? entry.getTargetPath()
+        .toByteArray() : null);
+  }
+
+  public static SnapshottableDirectoryStatus[] convert(
+      SnapshottableDirectoryListingProto sdlp) {
+    if (sdlp == null)
+      return null;
+    List<SnapshottableDirectoryStatusProto> list = sdlp
+        .getSnapshottableDirListingList();
+    if (list.isEmpty()) {
+      return new SnapshottableDirectoryStatus[0];
+    } else {
+      SnapshottableDirectoryStatus[] result =
+          new SnapshottableDirectoryStatus[list.size()];
+      for (int i = 0; i < list.size(); i++) {
+        result[i] = convert(list.get(i));
+      }
+      return result;
+    }
+  }
+
+  public static SnapshottableDirectoryStatus convert(
+      SnapshottableDirectoryStatusProto sdirStatusProto) {
+    if (sdirStatusProto == null) {
+      return null;
+    }
+    final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
+    return new SnapshottableDirectoryStatus(
+        status.getModificationTime(),
+        status.getAccessTime(),
+        convert(status.getPermission()),
+        status.getOwner(),
+        status.getGroup(),
+        status.getPath().toByteArray(),
+        status.getFileId(),
+        status.getChildrenNum(),
+        sdirStatusProto.getSnapshotNumber(),
+        sdirStatusProto.getSnapshotQuota(),
+        sdirStatusProto.getParentFullpath().toByteArray());
+  }
+
+  // DataEncryptionKey
+  public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
+    String encryptionAlgorithm = bet.getEncryptionAlgorithm();
+    return new DataEncryptionKey(bet.getKeyId(),
+        bet.getBlockPoolId(),
+        bet.getNonce().toByteArray(),
+        bet.getEncryptionKey().toByteArray(),
+        bet.getExpiryDate(),
+        encryptionAlgorithm.isEmpty() ? null : encryptionAlgorithm);
+  }
+
+  public static Token<DelegationTokenIdentifier> convertDelegationToken(
+      TokenProto blockToken) {
+    return new Token<>(blockToken.getIdentifier()
+        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+        blockToken.getKind()), new Text(blockToken.getService()));
+  }
+
+  // Arrays of DatanodeId
+  public static DatanodeIDProto[] convert(DatanodeID[] did) {
+    if (did == null)
+      return null;
+    final int len = did.length;
+    DatanodeIDProto[] result = new DatanodeIDProto[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(did[i]);
+    }
+    return result;
+  }
+
+  public static FsPermissionProto convert(FsPermission p) {
+    return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
+  }
+
+  public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
+    if (fs == null)
+      return null;
+    return new HdfsLocatedFileStatus(
+        fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
+        fs.getBlockReplication(), fs.getBlocksize(),
+        fs.getModificationTime(), fs.getAccessTime(),
+        convert(fs.getPermission()), fs.getOwner(), fs.getGroup(),
+        fs.getFileType().equals(FileType.IS_SYMLINK) ?
+            fs.getSymlink().toByteArray() : null,
+        fs.getPath().toByteArray(),
+        fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
+        fs.hasLocations() ? convert(fs.getLocations()) : null,
+        fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
+        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
+        fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+  }
+
+  public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
+    if (c == null)
+      return null;
+    List<String> fileList = c.getFilesList();
+    return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
+        c.getCookie());
+  }
+
+  public static ContentSummary convert(ContentSummaryProto cs) {
+    if (cs == null) return null;
+    ContentSummary.Builder builder = new ContentSummary.Builder();
+    builder.length(cs.getLength()).
+        fileCount(cs.getFileCount()).
+        directoryCount(cs.getDirectoryCount()).
+        quota(cs.getQuota()).
+        spaceConsumed(cs.getSpaceConsumed()).
+        spaceQuota(cs.getSpaceQuota());
+    if (cs.hasTypeQuotaInfos()) {
+      for (HdfsProtos.StorageTypeQuotaInfoProto info :
+          cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
+        StorageType type = convertStorageType(info.getType());
+        builder.typeConsumed(type, info.getConsumed());
+        builder.typeQuota(type, info.getQuota());
+      }
+    }
+    return builder.build();
+  }
+
+  public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
+    switch (a) {
+    case QUERY:
+      return RollingUpgradeActionProto.QUERY;
+    case PREPARE:
+      return RollingUpgradeActionProto.START;
+    case FINALIZE:
+      return RollingUpgradeActionProto.FINALIZE;
+    default:
+      throw new IllegalArgumentException("Unexpected value: " + a);
+    }
+  }
+
+  public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
+    RollingUpgradeStatusProto status = proto.getStatus();
+    return new RollingUpgradeInfo(status.getBlockPoolId(),
+        proto.getCreatedRollbackImages(),
+        proto.getStartTime(), proto.getFinalizeTime());
+  }
+
+  public static DatanodeStorageReport[] convertDatanodeStorageReports(
+      List<DatanodeStorageReportProto> protos) {
+    final DatanodeStorageReport[] reports
+        = new DatanodeStorageReport[protos.size()];
+    for(int i = 0; i < reports.length; i++) {
+      reports[i] = convertDatanodeStorageReport(protos.get(i));
+    }
+    return reports;
+  }
+
+  public static DatanodeStorageReport convertDatanodeStorageReport(
+      DatanodeStorageReportProto proto) {
+    return new DatanodeStorageReport(
+        convert(proto.getDatanodeInfo()),
+        convertStorageReports(proto.getStorageReportsList()));
+  }
+
+  public static StorageReport[] convertStorageReports(
+      List<StorageReportProto> list) {
+    final StorageReport[] report = new StorageReport[list.size()];
+    for (int i = 0; i < report.length; i++) {
+      report[i] = convert(list.get(i));
+    }
+    return report;
+  }
+
+  public static StorageReport convert(StorageReportProto p) {
+    return new StorageReport(
+        p.hasStorage() ?
+            convert(p.getStorage()) :
+            new DatanodeStorage(p.getStorageUuid()),
+        p.getFailed(), p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+        p.getBlockPoolUsed());
+  }
+
+  public static DatanodeStorage convert(DatanodeStorageProto s) {
+    return new DatanodeStorage(s.getStorageUuid(),
+        convertState(s.getState()), convertStorageType(s.getStorageType()));
+  }
+
+  private static State convertState(StorageState state) {
+    switch(state) {
+    case READ_ONLY_SHARED:
+      return State.READ_ONLY_SHARED;
+    case NORMAL:
+    default:
+      return State.NORMAL;
+    }
+  }
+
+  public static SafeModeActionProto convert(
+      SafeModeAction a) {
+    switch (a) {
+    case SAFEMODE_LEAVE:
+      return SafeModeActionProto.SAFEMODE_LEAVE;
+    case SAFEMODE_ENTER:
+      return SafeModeActionProto.SAFEMODE_ENTER;
+    case SAFEMODE_GET:
+      return SafeModeActionProto.SAFEMODE_GET;
+    default:
+      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+    }
+  }
+
+  public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
+    DatanodeInfo[] info = new DatanodeInfo[list.size()];
+    for (int i = 0; i < info.length; i++) {
+      info[i] = convert(list.get(i));
+    }
+    return info;
+  }
+
+  public static long[] convert(GetFsStatsResponseProto res) {
+    long[] result = new long[7];
+    result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
+    result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
+    result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
+    result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
+    result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
+    result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
+    result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
+        res.getMissingReplOneBlocks();
+    return result;
+  }
+
+  public static DatanodeReportTypeProto
+    convert(DatanodeReportType t) {
+    switch (t) {
+    case ALL: return DatanodeReportTypeProto.ALL;
+    case LIVE: return DatanodeReportTypeProto.LIVE;
+    case DEAD: return DatanodeReportTypeProto.DEAD;
+    case DECOMMISSIONING: return DatanodeReportTypeProto.DECOMMISSIONING;
+    default:
+      throw new IllegalArgumentException("Unexpected data type report:" + t);
+    }
+  }
+
+  public static DirectoryListing convert(DirectoryListingProto dl) {
+    if (dl == null)
+      return null;
+    List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
+    return new DirectoryListing(partList.isEmpty() ?
+        new HdfsLocatedFileStatus[0] :
+        convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
+        dl.getRemainingEntries());
+  }
+
+  public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
+    if (fs == null) return null;
+    final int len = fs.length;
+    HdfsFileStatus[] result = new HdfsFileStatus[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(fs[i]);
+    }
+    return result;
+  }
+
+  // The creatFlag field in PB is a bitmask whose values are the same a the
+  // emum values of CreateFlag
+  public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
+    int value = 0;
+    if (flag.contains(CreateFlag.APPEND)) {
+      value |= CreateFlagProto.APPEND.getNumber();
+    }
+    if (flag.contains(CreateFlag.CREATE)) {
+      value |= CreateFlagProto.CREATE.getNumber();
+    }
+    if (flag.contains(CreateFlag.OVERWRITE)) {
+      value |= CreateFlagProto.OVERWRITE.getNumber();
+    }
+    if (flag.contains(CreateFlag.LAZY_PERSIST)) {
+      value |= CreateFlagProto.LAZY_PERSIST.getNumber();
+    }
+    if (flag.contains(CreateFlag.NEW_BLOCK)) {
+      value |= CreateFlagProto.NEW_BLOCK.getNumber();
+    }
+    return value;
+  }
+
+  public static FsServerDefaults convert(FsServerDefaultsProto fs) {
+    if (fs == null) return null;
+    return new FsServerDefaults(
+        fs.getBlockSize(), fs.getBytesPerChecksum(),
+        fs.getWritePacketSize(), (short) fs.getReplication(),
+        fs.getFileBufferSize(),
+        fs.getEncryptDataTransfer(),
+        fs.getTrashInterval(),
+        convert(fs.getChecksumType()));
+  }
+
+  public static List<CryptoProtocolVersionProto> convert(
+      CryptoProtocolVersion[] versions) {
+    List<CryptoProtocolVersionProto> protos =
+        Lists.newArrayListWithCapacity(versions.length);
+    for (CryptoProtocolVersion v: versions) {
+      protos.add(convert(v));
+    }
+    return protos;
+  }
+
+  static List<StorageTypesProto> convert(StorageType[][] types) {
+    List<StorageTypesProto> list = Lists.newArrayList();
+    if (types != null) {
+      for (StorageType[] ts : types) {
+        StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
+        builder.addAllStorageTypes(convertStorageTypes(ts));
+        list.add(builder.build());
+      }
+    }
+    return list;
+  }
+
+  public static BlockStoragePolicyProto convert(BlockStoragePolicy policy) {
+    BlockStoragePolicyProto.Builder builder = BlockStoragePolicyProto
+        .newBuilder().setPolicyId(policy.getId()).setName(policy.getName());
+    // creation storage types
+    StorageTypesProto creationProto = convert(policy.getStorageTypes());
+    Preconditions.checkArgument(creationProto != null);
+    builder.setCreationPolicy(creationProto);
+    // creation fallback
+    StorageTypesProto creationFallbackProto = convert(
+        policy.getCreationFallbacks());
+    if (creationFallbackProto != null) {
+      builder.setCreationFallbackPolicy(creationFallbackProto);
+    }
+    // replication fallback
+    StorageTypesProto replicationFallbackProto = convert(
+        policy.getReplicationFallbacks());
+    if (replicationFallbackProto != null) {
+      builder.setReplicationFallbackPolicy(replicationFallbackProto);
+    }
+    return builder.build();
+  }
+
+  public static StorageTypesProto convert(StorageType[] types) {
+    if (types == null || types.length == 0) {
+      return null;
+    }
+    List<StorageTypeProto> list = convertStorageTypes(types);
+    return StorageTypesProto.newBuilder().addAllStorageTypes(list).build();
+  }
+
+  public static DatanodeID[] convert(DatanodeIDProto[] did) {
+    if (did == null) return null;
+    final int len = did.length;
+    DatanodeID[] result = new DatanodeID[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(did[i]);
+    }
+    return result;
+  }
+
+  // Block
+  public static BlockProto convert(Block b) {
+    return BlockProto.newBuilder().setBlockId(b.getBlockId())
+        .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
+        .build();
+  }
+
+  public static Block convert(BlockProto b) {
+    return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
+  }
+
+  static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
+    if (di == null) return null;
+    DatanodeInfo[] result = new DatanodeInfo[di.length];
+    for (int i = 0; i < di.length; i++) {
+      result[i] = convert(di[i]);
+    }
+    return result;
+  }
+
+  public static DatanodeStorageReportProto convertDatanodeStorageReport(
+      DatanodeStorageReport report) {
+    return DatanodeStorageReportProto.newBuilder()
+        .setDatanodeInfo(convert(report.getDatanodeInfo()))
+        .addAllStorageReports(convertStorageReports(report.getStorageReports()))
+        .build();
+  }
+
+  public static List<DatanodeStorageReportProto> convertDatanodeStorageReports(
+      DatanodeStorageReport[] reports) {
+    final List<DatanodeStorageReportProto> protos
+        = new ArrayList<>(reports.length);
+    for(int i = 0; i < reports.length; i++) {
+      protos.add(convertDatanodeStorageReport(reports[i]));
+    }
+    return protos;
+  }
+
+  public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
+    if (lb == null) return null;
+    return convertLocatedBlock(Arrays.asList(lb)).toArray(
+        new LocatedBlock[lb.length]);
+  }
+
+  public static LocatedBlocksProto convert(LocatedBlocks lb) {
+    if (lb == null) {
+      return null;
+    }
+    LocatedBlocksProto.Builder builder =
+        LocatedBlocksProto.newBuilder();
+    if (lb.getLastLocatedBlock() != null) {
+      builder.setLastBlock(convert(lb.getLastLocatedBlock()));
+    }
+    if (lb.getFileEncryptionInfo() != null) {
+      builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
+    }
+    return builder.setFileLength(lb.getFileLength())
+        .setUnderConstruction(lb.isUnderConstruction())
+        .addAllBlocks(convertLocatedBlock2(lb.getLocatedBlocks()))
+        .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
+  }
+
+  public static DataEncryptionKeyProto convert(DataEncryptionKey bet) {
+    DataEncryptionKeyProto.Builder b = DataEncryptionKeyProto.newBuilder()
+        .setKeyId(bet.keyId)
+        .setBlockPoolId(bet.blockPoolId)
+        .setNonce(ByteString.copyFrom(bet.nonce))
+        .setEncryptionKey(ByteString.copyFrom(bet.encryptionKey))
+        .setExpiryDate(bet.expiryDate);
+    if (bet.encryptionAlgorithm != null) {
+      b.setEncryptionAlgorithm(bet.encryptionAlgorithm);
+    }
+    return b.build();
+  }
+
+  public static FsServerDefaultsProto convert(FsServerDefaults fs) {
+    if (fs == null) return null;
+    return FsServerDefaultsProto.newBuilder().
+      setBlockSize(fs.getBlockSize()).
+      setBytesPerChecksum(fs.getBytesPerChecksum()).
+      setWritePacketSize(fs.getWritePacketSize())
+      .setReplication(fs.getReplication())
+      .setFileBufferSize(fs.getFileBufferSize())
+      .setEncryptDataTransfer(fs.getEncryptDataTransfer())
+      .setTrashInterval(fs.getTrashInterval())
+      .setChecksumType(convert(fs.getChecksumType()))
+      .build();
+  }
+
+  public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
+    EnumSet<CreateFlag> result =
+       EnumSet.noneOf(CreateFlag.class);
+    if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
+      result.add(CreateFlag.APPEND);
+    }
+    if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
+      result.add(CreateFlag.CREATE);
+    }
+    if ((flag & CreateFlagProto.OVERWRITE_VALUE)
+        == CreateFlagProto.OVERWRITE_VALUE) {
+      result.add(CreateFlag.OVERWRITE);
+    }
+    if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
+        == CreateFlagProto.LAZY_PERSIST_VALUE) {
+      result.add(CreateFlag.LAZY_PERSIST);
+    }
+    if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
+        == CreateFlagProto.NEW_BLOCK_VALUE) {
+      result.add(CreateFlag.NEW_BLOCK);
+    }
+    return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
+  }
+
+  public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
+    EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
+    if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
+      result.add(CacheFlag.FORCE);
+    }
+    return result;
+  }
+
+  public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
+    if (fs == null)
+      return null;
+    FileType fType = FileType.IS_FILE;
+    if (fs.isDir()) {
+      fType = FileType.IS_DIR;
+    } else if (fs.isSymlink()) {
+      fType = FileType.IS_SYMLINK;
+    }
+
+    HdfsFileStatusProto.Builder builder =
+     HdfsFileStatusProto.newBuilder().
+      setLength(fs.getLen()).
+      setFileType(fType).
+      setBlockReplication(fs.getReplication()).
+      setBlocksize(fs.getBlockSize()).
+      setModificationTime(fs.getModificationTime()).
+      setAccessTime(fs.getAccessTime()).
+      setPermission(convert(fs.getPermission())).
+      setOwner(fs.getOwner()).
+      setGroup(fs.getGroup()).
+      setFileId(fs.getFileId()).
+      setChildrenNum(fs.getChildrenNum()).
+      setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
+      setStoragePolicy(fs.getStoragePolicy());
+    if (fs.isSymlink())  {
+      builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
+    }
+    if (fs.getFileEncryptionInfo() != null) {
+      builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
+    }
+    if (fs instanceof HdfsLocatedFileStatus) {
+      final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
+      LocatedBlocks locations = lfs.getBlockLocations();
+      if (locations != null) {
+        builder.setLocations(convert(locations));
+      }
+    }
+    return builder.build();
+  }
+
+  public static SnapshottableDirectoryStatusProto convert(
+      SnapshottableDirectoryStatus status) {
+    if (status == null) {
+      return null;
+    }
+    int snapshotNumber = status.getSnapshotNumber();
+    int snapshotQuota = status.getSnapshotQuota();
+    byte[] parentFullPath = status.getParentFullPath();
+    ByteString parentFullPathBytes = ByteString.copyFrom(
+        parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
+    HdfsFileStatusProto fs = convert(status.getDirStatus());
+    SnapshottableDirectoryStatusProto.Builder builder =
+        SnapshottableDirectoryStatusProto
+        .newBuilder().setSnapshotNumber(snapshotNumber)
+        .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
+        .setDirStatus(fs);
+    return builder.build();
+  }
+
+  public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
+    if (fs == null) return null;
+    final int len = fs.length;
+    HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(fs[i]);
+    }
+    return result;
+  }
+
+  public static DirectoryListingProto convert(DirectoryListing d) {
+    if (d == null)
+      return null;
+    return DirectoryListingProto.newBuilder().
+        addAllPartialListing(Arrays.asList(
+            convert(d.getPartialListing()))).
+        setRemainingEntries(d.getRemainingEntries()).
+        build();
+  }
+
+  public static GetFsStatsResponseProto convert(long[] fsStats) {
+    GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
+        .newBuilder();
+    if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
+      result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
+      result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
+      result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
+      result.setUnderReplicated(
+              fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
+      result.setCorruptBlocks(
+          fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
+      result.setMissingBlocks(
+          fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
+      result.setMissingReplOneBlocks(
+          fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
+    return result.build();
+  }
+
+  public static DatanodeReportType convert(DatanodeReportTypeProto t) {
+    switch (t) {
+    case ALL: return DatanodeReportType.ALL;
+    case LIVE: return DatanodeReportType.LIVE;
+    case DEAD: return DatanodeReportType.DEAD;
+    case DECOMMISSIONING: return DatanodeReportType.DECOMMISSIONING;
+    default:
+      throw new IllegalArgumentException("Unexpected data type report:" + t);
+    }
+  }
+
+  public static SafeModeAction convert(
+      SafeModeActionProto a) {
+    switch (a) {
+    case SAFEMODE_LEAVE:
+      return SafeModeAction.SAFEMODE_LEAVE;
+    case SAFEMODE_ENTER:
+      return SafeModeAction.SAFEMODE_ENTER;
+    case SAFEMODE_GET:
+      return SafeModeAction.SAFEMODE_GET;
+    default:
+      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+    }
+  }
+
+  public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
+    switch (a) {
+    case QUERY:
+      return RollingUpgradeAction.QUERY;
+    case START:
+      return RollingUpgradeAction.PREPARE;
+    case FINALIZE:
+      return RollingUpgradeAction.FINALIZE;
+    default:
+      throw new IllegalArgumentException("Unexpected value: " + a);
+    }
+  }
+
+  public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
+      RollingUpgradeStatus status) {
+    return RollingUpgradeStatusProto.newBuilder()
+        .setBlockPoolId(status.getBlockPoolId())
+        .setFinalized(status.isFinalized())
+        .build();
+  }
+
+  public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
+    return new RollingUpgradeStatus(proto.getBlockPoolId(),
+        proto.getFinalized());
+  }
+
+  public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
+    return RollingUpgradeInfoProto.newBuilder()
+        .setStatus(convertRollingUpgradeStatus(info))
+        .setCreatedRollbackImages(info.createdRollbackImages())
+        .setStartTime(info.getStartTime())
+        .setFinalizeTime(info.getFinalizeTime())
+        .build();
+  }
+
+  public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
+    if (c == null)
+      return null;
+    return CorruptFileBlocksProto.newBuilder().
+        addAllFiles(Arrays.asList(c.getFiles())).
+        setCookie(c.getCookie()).
+        build();
+  }
+
+  public static ContentSummaryProto convert(ContentSummary cs) {
+    if (cs == null) return null;
+    ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
+        builder.setLength(cs.getLength()).
+        setFileCount(cs.getFileCount()).
+        setDirectoryCount(cs.getDirectoryCount()).
+        setQuota(cs.getQuota()).
+        setSpaceConsumed(cs.getSpaceConsumed()).
+        setSpaceQuota(cs.getSpaceQuota());
+
+    if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
+      HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
+          HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
+      for (StorageType t: StorageType.getTypesSupportingQuota()) {
+        HdfsProtos.StorageTypeQuotaInfoProto info =
+            HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
+                setType(convertStorageType(t)).
+                setConsumed(cs.getTypeConsumed(t)).
+                setQuota(cs.getTypeQuota(t)).
+                build();
+        isb.addTypeQuotaInfo(info);
+      }
+      builder.setTypeQuotaInfos(isb);
+    }
+    return builder.build();
+  }
+
+  public static DatanodeStorageProto convert(DatanodeStorage s) {
+    return DatanodeStorageProto.newBuilder()
+        .setState(convertState(s.getState()))
+        .setStorageType(convertStorageType(s.getStorageType()))
+        .setStorageUuid(s.getStorageID()).build();
+  }
+
+  private static StorageState convertState(State state) {
+    switch(state) {
+    case READ_ONLY_SHARED:
+      return StorageState.READ_ONLY_SHARED;
+    case NORMAL:
+    default:
+      return StorageState.NORMAL;
+    }
+  }
+
+  public static StorageReportProto convert(StorageReport r) {
+    StorageReportProto.Builder builder = StorageReportProto.newBuilder()
+        .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
+        .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
+        .setStorageUuid(r.getStorage().getStorageID())
+        .setStorage(convert(r.getStorage()));
+    return builder.build();
+  }
+
+  public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
+    final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
+        storages.length);
+    for(int i = 0; i < storages.length; i++) {
+      protos.add(convert(storages[i]));
+    }
+    return protos;
+  }
+
+  public static SnapshottableDirectoryListingProto convert(
+      SnapshottableDirectoryStatus[] status) {
+    if (status == null)
+      return null;
+    SnapshottableDirectoryStatusProto[] protos =
+        new SnapshottableDirectoryStatusProto[status.length];
+    for (int i = 0; i < status.length; i++) {
+      protos[i] = convert(status[i]);
+    }
+    List<SnapshottableDirectoryStatusProto> protoList = Arrays.asList(protos);
+    return SnapshottableDirectoryListingProto.newBuilder()
+        .addAllSnapshottableDirListing(protoList).build();
+  }
+
+  public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
+    if (entry == null) {
+      return null;
+    }
+    ByteString sourcePath = ByteString
+        .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+            .getSourcePath());
+    String modification = entry.getType().getLabel();
+    SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
+        .newBuilder().setFullpath(sourcePath)
+        .setModificationLabel(modification);
+    if (entry.getType() == DiffType.RENAME) {
+      ByteString targetPath = ByteString
+          .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+              .getTargetPath());
+      builder.setTargetPath(targetPath);
+    }
+    return builder.build();
+  }
+
+  public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
+    if (report == null) {
+      return null;
+    }
+    List<DiffReportEntry> entries = report.getDiffList();
+    List<SnapshotDiffReportEntryProto> entryProtos = new ArrayList<>();
+    for (DiffReportEntry entry : entries) {
+      SnapshotDiffReportEntryProto entryProto = convert(entry);
+      if (entryProto != null)
+        entryProtos.add(entryProto);
+    }
+
+    SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
+        .setSnapshotRoot(report.getSnapshotRoot())
+        .setFromSnapshot(report.getFromSnapshot())
+        .setToSnapshot(report.getLaterSnapshotName())
+        .addAllDiffReportEntries(entryProtos).build();
+    return reportProto;
+  }
+
+  public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
+    CacheDirectiveStatsProto.Builder builder =
+        CacheDirectiveStatsProto.newBuilder();
+    builder.setBytesNeeded(stats.getBytesNeeded());
+    builder.setBytesCached(stats.getBytesCached());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
+    builder.setHasExpired(stats.hasExpired());
+    return builder.build();
+  }
+
+  public static CacheDirectiveEntryProto convert(CacheDirectiveEntry entry) {
+    CacheDirectiveEntryProto.Builder builder =
+        CacheDirectiveEntryProto.newBuilder();
+    builder.setInfo(convert(entry.getInfo()));
+    builder.setStats(convert(entry.getStats()));
+    return builder.build();
+  }
+
+  public static boolean[] convertBooleanList(
+    List<Boolean> targetPinningsList) {
+    final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
+    for (int i = 0; i < targetPinningsList.size(); i++) {
+      targetPinnings[i] = targetPinningsList.get(i);
+    }
+    return targetPinnings;
+  }
+
+  public static CachePoolStatsProto convert(CachePoolStats stats) {
+    CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
+    builder.setBytesNeeded(stats.getBytesNeeded());
+    builder.setBytesCached(stats.getBytesCached());
+    builder.setBytesOverlimit(stats.getBytesOverlimit());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
+    return builder.build();
+  }
+
+  public static CachePoolEntryProto convert(CachePoolEntry entry) {
+    CachePoolEntryProto.Builder builder = CachePoolEntryProto.newBuilder();
+    builder.setInfo(convert(entry.getInfo()));
+    builder.setStats(convert(entry.getStats()));
+    return builder.build();
+  }
+
+  public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
+    DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
+    builder.setSoftwareVersion(info.getSoftwareVersion());
+    builder.setConfigVersion(info.getConfigVersion());
+    builder.setUptime(info.getUptime());
+    return builder.build();
+  }
+
+  public static GetAclStatusResponseProto convert(AclStatus e) {
+    AclStatusProto.Builder builder = AclStatusProto.newBuilder();
+    builder.setOwner(e.getOwner())
+        .setGroup(e.getGroup()).setSticky(e.isStickyBit())
+        .addAllEntries(convertAclEntryProto(e.getEntries()));
+    if (e.getPermission() != null) {
+      builder.setPermission(convert(e.getPermission()));
+    }
+    AclStatusProto r = builder.build();
+    return GetAclStatusResponseProto.newBuilder().setResult(r).build();
+  }
+
+  public static EnumSet<XAttrSetFlag> convert(int flag) {
+    EnumSet<XAttrSetFlag> result =
+        EnumSet.noneOf(XAttrSetFlag.class);
+    if ((flag & XAttrSetFlagProto.XATTR_CREATE_VALUE) ==
+        XAttrSetFlagProto.XATTR_CREATE_VALUE) {
+      result.add(XAttrSetFlag.CREATE);
+    }
+    if ((flag & XAttrSetFlagProto.XATTR_REPLACE_VALUE) ==
+        XAttrSetFlagProto.XATTR_REPLACE_VALUE) {
+      result.add(XAttrSetFlag.REPLACE);
+    }
+    return result;
+  }
+
+  public static XAttr convertXAttr(XAttrProto a) {
+    XAttr.Builder builder = new XAttr.Builder();
+    builder.setNameSpace(convert(a.getNamespace()));
+    if (a.hasName()) {
+      builder.setName(a.getName());
+    }
+    if (a.hasValue()) {
+      builder.setValue(a.getValue().toByteArray());
+    }
+    return builder.build();
+  }
+
+  public static GetXAttrsResponseProto convertXAttrsResponse(
+      List<XAttr> xAttrs) {
+    GetXAttrsResponseProto.Builder builder = GetXAttrsResponseProto
+        .newBuilder();
+    if (xAttrs != null) {
+      builder.addAllXAttrs(convertXAttrProto(xAttrs));
+    }
+    return builder.build();
+  }
+
+  public static ListXAttrsResponseProto convertListXAttrsResponse(
+    List<XAttr> names) {
+    ListXAttrsResponseProto.Builder builder =
+      ListXAttrsResponseProto.newBuilder();
+    if (names != null) {
+      builder.addAllXAttrs(convertXAttrProto(names));
+    }
+    return builder.build();
+  }
+
+  public static EncryptionZoneProto convert(EncryptionZone zone) {
+    return EncryptionZoneProto.newBuilder()
+        .setId(zone.getId())
+        .setPath(zone.getPath())
+        .setSuite(convert(zone.getSuite()))
+        .setCryptoProtocolVersion(convert(zone.getVersion()))
+        .setKeyName(zone.getKeyName())
+        .build();
+  }
+
+  public static SlotId convert(ShortCircuitShmSlotProto slotId) {
+    return new SlotId(convert(slotId.getShmId()),
+        slotId.getSlotIdx());
+  }
+
+  public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
+    InotifyProtos.EventsListProto.Builder builder =
+        InotifyProtos.EventsListProto.newBuilder();
+    for (EventBatch b : el.getBatches()) {
+      List<InotifyProtos.EventProto> events = Lists.newArrayList();
+      for (Event e : b.getEvents()) {
+        switch (e.getEventType()) {
+          case CLOSE:
+            Event.CloseEvent ce = (Event.CloseEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_CLOSE)
+                .setContents(
+                    InotifyProtos.CloseEventProto.newBuilder()
+                        .setPath(ce.getPath())
+                        .setFileSize(ce.getFileSize())
+                        .setTimestamp(ce.getTimestamp()).build().toByteString()
+                ).build());
+            break;
+          case CREATE:
+            Event.CreateEvent ce2 = (Event.CreateEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_CREATE)
+                .setContents(
+                    InotifyProtos.CreateEventProto.newBuilder()
+                        .setType(createTypeConvert(ce2.getiNodeType()))
+                        .setPath(ce2.getPath())
+                        .setCtime(ce2.getCtime())
+                        .setOwnerName(ce2.getOwnerName())
+                        .setGroupName(ce2.getGroupName())
+                        .setPerms(convert(ce2.getPerms()))
+                        .setReplication(ce2.getReplication())
+                        .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
+                            "" : ce2.getSymlinkTarget())
+                        .setDefaultBlockSize(ce2.getDefaultBlockSize())
+                        .setOverwrite(ce2.getOverwrite()).build().toByteString()
+                ).build());
+            break;
+          case METADATA:
+            Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
+            InotifyProtos.MetadataUpdateEventProto.Builder metaB =
+                InotifyProtos.MetadataUpdateEventProto.newBuilder()
+                    .setPath(me.getPath())
+                    .setType(metadataUpdateTypeConvert(me.getMetadataType()))
+                    .setMtime(me.getMtime())
+                    .setAtime(me.getAtime())
+                    .setReplication(me.getReplication())
+                    .setOwnerName(me.getOwnerName() == null ? "" :
+                        me.getOwnerName())
+                    .setGroupName(me.getGroupName() == null ? "" :
+                        me.getGroupName())
+                    .addAllAcls(me.getAcls() == null ?
+                        Lists.<AclEntryProto>newArrayList() :
+                        convertAclEntryProto(me.getAcls()))
+                    .addAllXAttrs(me.getxAttrs() == null ?
+                        Lists.<XAttrProto>newArrayList() :
+                        convertXAttrProto(me.getxAttrs()))
+                    .setXAttrsRemoved(me.isxAttrsRemoved());
+            if (me.getPerms() != null) {
+              metaB.setPerms(convert(me.getPerms()));
+            }
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_METADATA)
+                .setContents(metaB.build().toByteString())
+                .build());
+            break;
+          case RENAME:
+            Event.RenameEvent re = (Event.RenameEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_RENAME)
+                .setContents(
+                    InotifyProtos.RenameEventProto.newBuilder()
+                        .setSrcPath(re.getSrcPath())
+                        .setDestPath(re.getDstPath())
+                        .setTimestamp(re.getTimestamp()).build().toByteString()
+                ).build());
+            break;
+          case APPEND:
+            Event.AppendEvent re2 = (Event.AppendEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_APPEND)
+                .setContents(InotifyProtos.AppendEventProto.newBuilder()
+                    .setPath(re2.getPath())
+                    .setNewBlock(re2.toNewBlock()).build().toByteString())
+                .build());
+            break;
+          case UNLINK:
+            Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_UNLINK)
+                .setContents(
+                    InotifyProtos.UnlinkEventProto.newBuilder()
+                        .setPath(ue.getPath())
+                        .setTimestamp(ue.getTimestamp()).build().toByteString()
+                ).build());
+            break;
+          case TRUNCATE:
+            Event.TruncateEvent te = (Event.TruncateEvent) e;
+            events.add(InotifyProtos.EventProto.newBuilder()
+                .setType(InotifyProtos.EventType.EVENT_TRUNCATE)
+                .setContents(
+                    InotifyProtos.TruncateEventProto.newBuilder()
+                        .setPath(te.getPath())
+                        .setFileSize(te.getFileSize())
+                        .setTimestamp(te.getTimestamp()).build().toByteString()
+                ).build());
+            break;
+          default:
+            throw new RuntimeException("Unexpected inotify event: " + e);
+        }
+      }
+      builder.addBatch(InotifyProtos.EventBatchProto.newBuilder().
+          setTxid(b.getTxid()).
+          addAllEvents(events));
+    }
+    builder.setFirstTxid(el.getFirstTxid());
+    builder.setLastTxid(el.getLastTxid());
+    builder.setSyncTxid(el.getSyncTxid());
+    return GetEditsFromTxidResponseProto.newBuilder().setEventsList(
+        builder.build()).build();
+  }
+
+  public static CryptoProtocolVersion[] convertCryptoProtocolVersions(
+      List<CryptoProtocolVersionProto> protos) {
+    List<CryptoProtocolVersion> versions =
+        Lists.newArrayListWithCapacity(protos.size());
+    for (CryptoProtocolVersionProto p: protos) {
+      versions.add(convert(p));
+    }
+    return versions.toArray(new CryptoProtocolVersion[]{});
+  }
+
+  public static HdfsProtos.PerFileEncryptionInfoProto convertPerFileEncInfo(
+      FileEncryptionInfo info) {
+    if (info == null) {
+      return null;
+    }
+    return HdfsProtos.PerFileEncryptionInfoProto.newBuilder()
+        .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+        .setIv(getByteString(info.getIV()))
+        .setEzKeyVersionName(info.getEzKeyVersionName())
+        .build();
+  }
+
+  public static HdfsProtos.ZoneEncryptionInfoProto convert(
+      CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+    if (suite == null || version == null || keyName == null) {
+      return null;
+    }
+    return HdfsProtos.ZoneEncryptionInfoProto.newBuilder()
+        .setSuite(convert(suite))
+        .setCryptoProtocolVersion(convert(version))
+        .setKeyName(keyName)
+        .build();
+  }
+
+  public static FileEncryptionInfo convert(
+      HdfsProtos.PerFileEncryptionInfoProto fileProto,
+      CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+    if (fileProto == null || suite == null || version == null ||
+        keyName == null) {
+      return null;
+    }
+    byte[] key = fileProto.getKey().toByteArray();
+    byte[] iv = fileProto.getIv().toByteArray();
+    String ezKeyVersionName = fileProto.getEzKeyVersionName();
+    return new FileEncryptionInfo(suite, version, key, iv, keyName,
+        ezKeyVersionName);
+  }
+
+  public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
+    List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
+    DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
+    for (int i = 0; i < infos.length; i++) {
+      infos[i] = convert(proto.get(i));
+    }
+    return infos;
+  }
+
+  static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
+    DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
+    for (int i = 0; i < targets.length; i++) {
+      ret[i] = DatanodeInfosProto.newBuilder()
+          .addAllDatanodes(convert(targets[i])).build();
+    }
+    return Arrays.asList(ret);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06022b8f/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 65cde45..944986c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -938,6 +938,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9101. Remove deprecated NameNode.getUri() static helper method.
     (Mingliang Liu via wheat9)
 
+    HDFS-9111. Move hdfs-client protobuf convert methods from PBHelper to
+    PBHelperClient. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than