Browse Source

Merge trunk into HA branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1242635 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 years ago
parent
commit
c14bb61a82
100 changed files with 1530 additions and 6655 deletions
  1. 8 0
      hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
  2. 299 0
      hadoop-client/pom.xml
  3. 68 6
      hadoop-common-project/hadoop-common/CHANGES.txt
  4. 18 1
      hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
  5. 6 3
      hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh
  6. 58 14
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  7. 9 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  8. 33 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
  9. 9 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java
  10. 13 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
  11. 2 29
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
  12. 0 14
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
  13. 30 67
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
  14. 15 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
  15. 10 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java
  16. 22 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  17. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java
  18. 54 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java
  19. 97 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellCopy.java
  20. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java
  21. 102 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemDelegation.java
  22. 77 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java
  23. 117 31
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  24. 3 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh
  25. 2 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh
  26. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/hdfs_user_guide.xml
  27. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  28. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  30. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  31. 1 36
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  32. 76 104
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  33. 11 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
  34. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  35. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
  37. 12 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  38. 11 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  39. 59 30
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  40. 46 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
  41. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
  42. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
  43. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
  44. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
  45. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  46. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
  47. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
  48. 0 111
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java
  49. 0 129
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java
  50. 0 92
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CheckpointSignatureWritable.java
  51. 0 129
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolServerSideTranslatorR23.java
  52. 0 162
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolTranslatorR23.java
  53. 0 100
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeWireProtocol.java
  54. 0 463
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolServerSideTranslatorR23.java
  55. 0 480
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolTranslatorR23.java
  56. 0 478
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeWireProtocol.java
  57. 0 184
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ContentSummaryWritable.java
  58. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CorruptFileBlocksWritable.java
  59. 0 209
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeIDWritable.java
  60. 0 334
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeInfoWritable.java
  61. 0 157
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DirectoryListingWritable.java
  62. 0 104
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExportedBlockKeysWritable.java
  63. 0 121
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExtendedBlockWritable.java
  64. 0 80
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsPermissionWritable.java
  65. 0 122
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsServerDefaultsWritable.java
  66. 0 348
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsFileStatusWritable.java
  67. 0 116
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsLocatedFileStatusWritable.java
  68. 0 114
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolServerSideTranslatorR23.java
  69. 0 80
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolTranslatorR23.java
  70. 0 81
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalWireProtocol.java
  71. 0 253
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlockWritable.java
  72. 0 197
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlocksWritable.java
  73. 0 70
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeCommandWritable.java
  74. 0 163
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolServerSideTranslatorR23.java
  75. 0 179
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolTranslatorR23.java
  76. 0 98
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeRegistrationWritable.java
  77. 0 166
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java
  78. 0 100
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java
  79. 0 90
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java
  80. 0 69
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java
  81. 0 86
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/StorageInfoWritable.java
  82. 0 208
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/TokenWritable.java
  83. 0 140
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/UpgradeStatusReportWritable.java
  84. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
  85. 35 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  86. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  87. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  88. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  89. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  90. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  91. 19 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  92. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  93. 13 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  94. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  95. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  96. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockVolumeChoosingPolicy.java
  97. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  98. 9 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  99. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
  100. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

+ 8 - 0
hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml

@@ -71,6 +71,14 @@
       </includes>
       </includes>
       <fileMode>0755</fileMode>
       <fileMode>0755</fileMode>
     </fileSet>
     </fileSet>
+    <fileSet>
+      <directory>bin</directory>
+      <outputDirectory>sbin</outputDirectory>
+      <includes>
+        <include>mr-jobhistory-daemon.sh</include>
+      </includes>
+      <fileMode>0755</fileMode>
+    </fileSet>
     <fileSet>
     <fileSet>
       <directory>hadoop-yarn/conf</directory>
       <directory>hadoop-yarn/conf</directory>
       <outputDirectory>etc/hadoop</outputDirectory>
       <outputDirectory>etc/hadoop</outputDirectory>

+ 299 - 0
hadoop-client/pom.xml

@@ -0,0 +1,299 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>0.24.0-SNAPSHOT</version>
+    <relativePath>../hadoop-project</relativePath>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-client</artifactId>
+  <version>0.24.0-SNAPSHOT</version>
+  <packaging>jar</packaging>
+
+  <description>Apache Hadoop Client</description>
+  <name>Apache Hadoop Client</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-cli</groupId>
+          <artifactId>commons-cli</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-httpclient</groupId>
+          <artifactId>commons-httpclient</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>jsp-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>jetty</groupId>
+          <artifactId>org.mortbay.jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-api-2.1</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>servlet-api-2.5</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-json</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jdt</groupId>
+          <artifactId>core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.aspectj</groupId>
+          <artifactId>aspectjrt</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro-ipc</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.sf.kosmosfs</groupId>
+          <artifactId>kfs</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.java.dev.jets3t</groupId>
+          <artifactId>jets3t</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-daemon</groupId>
+          <artifactId>commons-daemon</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-app</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.cenqua.clover</groupId>
+          <artifactId>clover</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>jline</groupId>
+          <artifactId>jline</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject</groupId>
+          <artifactId>guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.contribs</groupId>
+          <artifactId>jersey-guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.cenqua.clover</groupId>
+          <artifactId>clover</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject</groupId>
+          <artifactId>guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.jersey-test-framework</groupId>
+          <artifactId>jersey-test-framework-grizzly2</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey</groupId>
+          <artifactId>jersey-server</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jersey.contribs</groupId>
+          <artifactId>jersey-guice</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.cenqua.clover</groupId>
+          <artifactId>clover</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.cenqua.clover</groupId>
+          <artifactId>clover</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+</project>
+

+ 68 - 6
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -13,6 +13,16 @@ Trunk (unreleased changes)
     (suresh)
     (suresh)
 
 
   IMPROVEMENTS
   IMPROVEMENTS
+    HADOOP-8048. Allow merging of Credentials (Daryn Sharp via tgraves)
+ 
+    HADOOP-8032. mvn site:stage-deploy should be able to use the scp protocol
+    to stage documents (Ravi Prakash via tgraves)
+
+    HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
+    not covered (Eric Charles via bobby)
+
+    HADOOP-8015. ChRootFileSystem should extend FilterFileSystem 
+    (Daryn Sharp via bobby)
 
 
     HADOOP-7595. Upgrade dependency to Avro 1.5.3. (Alejandro Abdelnur via atm)
     HADOOP-7595. Upgrade dependency to Avro 1.5.3. (Alejandro Abdelnur via atm)
 
 
@@ -79,12 +89,19 @@ Trunk (unreleased changes)
     HADOOP-7988. Upper case in hostname part of the principals doesn't work with 
     HADOOP-7988. Upper case in hostname part of the principals doesn't work with 
     kerberos. (jitendra)
     kerberos. (jitendra)
 
 
-  BUGS
-    HADOOP-7998. CheckFileSystem does not correctly honor setVerifyChecksum
-                 (Daryn Sharp via bobby)
+  BUG FIXES
 
 
-    HADOOP-7851. Configuration.getClasses() never returns the default value. 
-                 (Uma Maheswara Rao G via amarrk)
+    HADOOP-8018.  Hudson auto test for HDFS has started throwing javadoc
+    (Jon Eagles via bobby)
+
+    HADOOP-8001  ChecksumFileSystem's rename doesn't correctly handle checksum
+    files. (Daryn Sharp via bobby)
+
+    HADOOP-8006  TestFSInputChecker is failing in trunk.
+    (Daryn Sharp via bobby)
+
+    HADOOP-7998. CheckFileSystem does not correctly honor setVerifyChecksum
+    (Daryn Sharp via bobby)
 
 
     HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
     HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
     by Jersey (Alejandro Abdelnur via atm)
     by Jersey (Alejandro Abdelnur via atm)
@@ -137,7 +154,27 @@ Trunk (unreleased changes)
 
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
 
 
-Release 0.23.1 - Unreleased
+Release 0.23.2 - UNRELEASED 
+
+  NEW FEATURES                                                                    
+  
+  IMPROVEMENTS
+
+    HADOOP-7923. Automate the updating of version numbers in the doc system.
+    (szetszwo)
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HADOOP-8042  When copying a file out of HDFS, modifying it, and uploading
+    it back into HDFS, the put fails due to a CRC mismatch
+    (Daryn Sharp via bobby)
+
+    HADOOP-8035 Hadoop Maven site is inefficient and runs phases redundantly
+    (abayer via tucu)
+
+Release 0.23.1 - 2012-02-08 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
@@ -221,6 +258,14 @@ Release 0.23.1 - Unreleased
     HADOOP-8002. SecurityUtil acquired token message should be a debug rather than info.
     HADOOP-8002. SecurityUtil acquired token message should be a debug rather than info.
     (Arpit Gupta via mahadev)
     (Arpit Gupta via mahadev)
 
 
+    HADOOP-8009. Create hadoop-client and hadoop-minicluster artifacts for downstream 
+    projects. (tucu)
+
+    HADOOP-7470. Move up to Jackson 1.8.8.  (Enis Soztutar via szetszwo)
+
+    HADOOP-8027. Visiting /jmx on the daemon web interfaces may print
+    unnecessary error in logs. (atm)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -228,6 +273,14 @@ Release 0.23.1 - Unreleased
    HADOOP-7811. TestUserGroupInformation#testGetServerSideGroups test fails in chroot.
    HADOOP-7811. TestUserGroupInformation#testGetServerSideGroups test fails in chroot.
    (Jonathan Eagles via mahadev)
    (Jonathan Eagles via mahadev)
 
 
+   HADOOP-7813. Fix test-patch to use proper numerical comparison when checking
+   javadoc and findbugs warning counts. (Jonathan Eagles via tlipcon)
+
+   HADOOP-7841. Run tests with non-secure random. (tlipcon)
+
+    HADOOP-7851. Configuration.getClasses() never returns the default value. 
+                 (Uma Maheswara Rao G via amarrk)
+
    HADOOP-7787. Make source tarball use conventional name.
    HADOOP-7787. Make source tarball use conventional name.
    (Bruno Mahé via tomwhite)
    (Bruno Mahé via tomwhite)
 
 
@@ -311,6 +364,15 @@ Release 0.23.1 - Unreleased
    HADOOP-8000. fetchdt command not available in bin/hadoop.
    HADOOP-8000. fetchdt command not available in bin/hadoop.
    (Arpit Gupta via mahadev)
    (Arpit Gupta via mahadev)
 
 
+   HADOOP-7999. "hadoop archive" fails with ClassNotFoundException.
+   (Jason Lowe via mahadev)
+
+   HADOOP-8012. hadoop-daemon.sh and yarn-daemon.sh are trying to mkdir
+   and chown log/pid dirs which can fail. (Roman Shaposhnik via eli)
+
+   HADOOP-8013. ViewFileSystem does not honor setVerifyChecksum
+   (Daryn Sharp via bobby)
+
 Release 0.23.0 - 2011-11-01 
 Release 0.23.0 - 2011-11-01 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 18 - 1
hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh

@@ -20,6 +20,16 @@
 # Resolve links ($0 may be a softlink) and convert a relative path
 # Resolve links ($0 may be a softlink) and convert a relative path
 # to an absolute path.  NB: The -P option requires bash built-ins
 # to an absolute path.  NB: The -P option requires bash built-ins
 # or POSIX:2001 compliant cd and pwd.
 # or POSIX:2001 compliant cd and pwd.
+
+#   HADOOP_CLASSPATH Extra Java CLASSPATH entries.
+#
+#   HADOOP_USER_CLASSPATH_FIRST      When defined, the HADOOP_CLASSPATH is 
+#                                    added in the beginning of the global
+#                                    classpath. Can be defined, for example,
+#                                    by doing 
+#                                    export HADOOP_USER_CLASSPATH_FIRST=true
+#
+
 this="${BASH_SOURCE-$0}"
 this="${BASH_SOURCE-$0}"
 common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P)
 common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P)
 script="$(basename -- "$this")"
 script="$(basename -- "$this")"
@@ -153,6 +163,10 @@ fi
 # CLASSPATH initially contains $HADOOP_CONF_DIR
 # CLASSPATH initially contains $HADOOP_CONF_DIR
 CLASSPATH="${HADOOP_CONF_DIR}"
 CLASSPATH="${HADOOP_CONF_DIR}"
 
 
+if [ "$HADOOP_USER_CLASSPATH_FIRST" != "" ] && [ "$HADOOP_CLASSPATH" != "" ] ; then
+  CLASSPATH=${CLASSPATH}:${HADOOP_CLASSPATH}
+fi
+
 # so that filenames w/ spaces are handled correctly in loops below
 # so that filenames w/ spaces are handled correctly in loops below
 IFS=
 IFS=
 
 
@@ -174,7 +188,7 @@ fi
 CLASSPATH=${CLASSPATH}:$HADOOP_COMMON_HOME/$HADOOP_COMMON_DIR'/*'
 CLASSPATH=${CLASSPATH}:$HADOOP_COMMON_HOME/$HADOOP_COMMON_DIR'/*'
 
 
 # add user-specified CLASSPATH last
 # add user-specified CLASSPATH last
-if [ "$HADOOP_CLASSPATH" != "" ]; then
+if [ "$HADOOP_USER_CLASSPATH_FIRST" = "" ] && [ "$HADOOP_CLASSPATH" != "" ]; then
   CLASSPATH=${CLASSPATH}:${HADOOP_CLASSPATH}
   CLASSPATH=${CLASSPATH}:${HADOOP_CLASSPATH}
 fi
 fi
 
 
@@ -214,6 +228,9 @@ if [ -d "${HADOOP_PREFIX}/build/native" -o -d "${HADOOP_PREFIX}/$HADOOP_COMMON_L
   fi
   fi
 fi
 fi
 
 
+# setup a default TOOL_PATH
+TOOL_PATH="${TOOL_PATH:-$HADOOP_PREFIX/share/hadoop/tools/lib/*}"
+
 # cygwin path translation
 # cygwin path translation
 if $cygwin; then
 if $cygwin; then
   JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`
   JAVA_LIBRARY_PATH=`cygpath -p "$JAVA_LIBRARY_PATH"`

+ 6 - 3
hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh

@@ -95,8 +95,11 @@ fi
 if [ "$HADOOP_LOG_DIR" = "" ]; then
 if [ "$HADOOP_LOG_DIR" = "" ]; then
   export HADOOP_LOG_DIR="$HADOOP_PREFIX/logs"
   export HADOOP_LOG_DIR="$HADOOP_PREFIX/logs"
 fi
 fi
-mkdir -p "$HADOOP_LOG_DIR"
-chown $HADOOP_IDENT_STRING $HADOOP_LOG_DIR
+
+if [ ! -w "$HADOOP_LOG_DIR" ] ; then
+  mkdir -p "$HADOOP_LOG_DIR"
+  chown $HADOOP_IDENT_STRING $HADOOP_LOG_DIR
+fi
 
 
 if [ "$HADOOP_PID_DIR" = "" ]; then
 if [ "$HADOOP_PID_DIR" = "" ]; then
   HADOOP_PID_DIR=/tmp
   HADOOP_PID_DIR=/tmp
@@ -118,7 +121,7 @@ case $startStop in
 
 
   (start)
   (start)
 
 
-    mkdir -p "$HADOOP_PID_DIR"
+    [ -w "$HADOOP_PID_DIR" ] ||  mkdir -p "$HADOOP_PID_DIR"
 
 
     if [ -f $pid ]; then
     if [ -f $pid ]; then
       if kill -0 `cat $pid` > /dev/null 2>&1; then
       if kill -0 `cat $pid` > /dev/null 2>&1; then

+ 58 - 14
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -43,6 +43,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   private static final byte[] CHECKSUM_VERSION = new byte[] {'c', 'r', 'c', 0};
   private static final byte[] CHECKSUM_VERSION = new byte[] {'c', 'r', 'c', 0};
   private int bytesPerChecksum = 512;
   private int bytesPerChecksum = 512;
   private boolean verifyChecksum = true;
   private boolean verifyChecksum = true;
+  private boolean writeChecksum = true;
 
 
   public static double getApproxChkSumLength(long size) {
   public static double getApproxChkSumLength(long size) {
     return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
     return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
@@ -67,6 +68,11 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     this.verifyChecksum = verifyChecksum;
     this.verifyChecksum = verifyChecksum;
   }
   }
 
 
+  @Override
+  public void setWriteChecksum(boolean writeChecksum) {
+    this.writeChecksum = writeChecksum;
+  }
+  
   /** get the raw file system */
   /** get the raw file system */
   public FileSystem getRawFileSystem() {
   public FileSystem getRawFileSystem() {
     return fs;
     return fs;
@@ -119,7 +125,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     private static final int HEADER_LENGTH = 8;
     private static final int HEADER_LENGTH = 8;
     
     
     private int bytesPerSum = 1;
     private int bytesPerSum = 1;
-    private long fileLen = -1L;
     
     
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
       throws IOException {
       throws IOException {
@@ -244,6 +249,24 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       }
       }
       return nread;
       return nread;
     }
     }
+  }
+  
+  private static class FSDataBoundedInputStream extends FSDataInputStream {
+    private FileSystem fs;
+    private Path file;
+    private long fileLen = -1L;
+
+    FSDataBoundedInputStream(FileSystem fs, Path file, InputStream in)
+        throws IOException {
+      super(in);
+      this.fs = fs;
+      this.file = file;
+    }
+    
+    @Override
+    public boolean markSupported() {
+      return false;
+    }
     
     
     /* Return the file length */
     /* Return the file length */
     private long getFileLength() throws IOException {
     private long getFileLength() throws IOException {
@@ -304,9 +327,16 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
    */
    */
   @Override
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
-    return verifyChecksum
-      ? new FSDataInputStream(new ChecksumFSInputChecker(this, f, bufferSize))
-      : getRawFileSystem().open(f, bufferSize);
+    FileSystem fs;
+    InputStream in;
+    if (verifyChecksum) {
+      fs = this;
+      in = new ChecksumFSInputChecker(this, f, bufferSize);
+    } else {
+      fs = getRawFileSystem();
+      in = fs.open(f, bufferSize);
+    }
+    return new FSDataBoundedInputStream(fs, f, in);
   }
   }
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
@@ -404,9 +434,20 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         throw new IOException("Mkdirs failed to create " + parent);
         throw new IOException("Mkdirs failed to create " + parent);
       }
       }
     }
     }
-    final FSDataOutputStream out = new FSDataOutputStream(
-        new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
-            blockSize, progress), null);
+    final FSDataOutputStream out;
+    if (writeChecksum) {
+      out = new FSDataOutputStream(
+          new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
+              blockSize, progress), null);
+    } else {
+      out = fs.create(f, permission, overwrite, bufferSize, replication,
+          blockSize, progress);
+      // remove the checksum file since we aren't writing one
+      Path checkFile = getChecksumFile(f);
+      if (fs.exists(checkFile)) {
+        fs.delete(checkFile, true);
+      }
+    }
     if (permission != null) {
     if (permission != null) {
       setPermission(f, permission);
       setPermission(f, permission);
     }
     }
@@ -450,18 +491,21 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     if (fs.isDirectory(src)) {
     if (fs.isDirectory(src)) {
       return fs.rename(src, dst);
       return fs.rename(src, dst);
     } else {
     } else {
+      if (fs.isDirectory(dst)) {
+        dst = new Path(dst, src.getName());
+      }
 
 
       boolean value = fs.rename(src, dst);
       boolean value = fs.rename(src, dst);
       if (!value)
       if (!value)
         return false;
         return false;
 
 
-      Path checkFile = getChecksumFile(src);
-      if (fs.exists(checkFile)) { //try to rename checksum
-        if (fs.isDirectory(dst)) {
-          value = fs.rename(checkFile, dst);
-        } else {
-          value = fs.rename(checkFile, getChecksumFile(dst));
-        }
+      Path srcCheckFile = getChecksumFile(src);
+      Path dstCheckFile = getChecksumFile(dst);
+      if (fs.exists(srcCheckFile)) { //try to rename checksum
+        value = fs.rename(srcCheckFile, dstCheckFile);
+      } else if (fs.exists(dstCheckFile)) {
+        // no src checksum, so remove dst checksum
+        value = fs.delete(dstCheckFile, true); 
       }
       }
 
 
       return value;
       return value;

+ 9 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -1936,6 +1936,15 @@ public abstract class FileSystem extends Configured implements Closeable {
     //doesn't do anything
     //doesn't do anything
   }
   }
 
 
+  /**
+   * Set the write checksum flag. This is only applicable if the 
+   * corresponding FileSystem supports checksum. By default doesn't do anything.
+   * @param writeChecksum
+   */
+  public void setWriteChecksum(boolean writeChecksum) {
+    //doesn't do anything
+  }
+
   /**
   /**
    * Return a list of file status objects that corresponds to the list of paths
    * Return a list of file status objects that corresponds to the list of paths
    * excluding those non-existent paths.
    * excluding those non-existent paths.

+ 33 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 
 
 import java.io.*;
 import java.io.*;
 import java.net.URI;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 
 
@@ -51,6 +52,7 @@ import org.apache.hadoop.util.Progressable;
 public class FilterFileSystem extends FileSystem {
 public class FilterFileSystem extends FileSystem {
   
   
   protected FileSystem fs;
   protected FileSystem fs;
+  private String swapScheme;
   
   
   /*
   /*
    * so that extending classes can define it
    * so that extending classes can define it
@@ -63,13 +65,25 @@ public class FilterFileSystem extends FileSystem {
     this.statistics = fs.statistics;
     this.statistics = fs.statistics;
   }
   }
 
 
+  /**
+   * Get the raw file system 
+   * @return FileSystem being filtered
+   */
+  public FileSystem getRawFileSystem() {
+    return fs;
+  }
+
   /** Called after a new FileSystem instance is constructed.
   /** Called after a new FileSystem instance is constructed.
    * @param name a uri whose authority section names the host, port, etc.
    * @param name a uri whose authority section names the host, port, etc.
    *   for this FileSystem
    *   for this FileSystem
    * @param conf the configuration
    * @param conf the configuration
    */
    */
   public void initialize(URI name, Configuration conf) throws IOException {
   public void initialize(URI name, Configuration conf) throws IOException {
-    fs.initialize(name, conf);
+    super.initialize(name, conf);
+    String scheme = name.getScheme();
+    if (!scheme.equals(fs.getUri().getScheme())) {
+      swapScheme = scheme;
+    }
   }
   }
 
 
   /** Returns a URI whose scheme and authority identify this FileSystem.*/
   /** Returns a URI whose scheme and authority identify this FileSystem.*/
@@ -88,7 +102,19 @@ public class FilterFileSystem extends FileSystem {
   
   
   /** Make sure that a path specifies a FileSystem. */
   /** Make sure that a path specifies a FileSystem. */
   public Path makeQualified(Path path) {
   public Path makeQualified(Path path) {
-    return fs.makeQualified(path);
+    Path fqPath = fs.makeQualified(path);
+    // swap in our scheme if the filtered fs is using a different scheme
+    if (swapScheme != null) {
+      try {
+        // NOTE: should deal with authority, but too much other stuff is broken 
+        fqPath = new Path(
+            new URI(swapScheme, fqPath.toUri().getSchemeSpecificPart(), null)
+        );
+      } catch (URISyntaxException e) {
+        throw new IllegalArgumentException(e);
+      }
+    }
+    return fqPath;
   }
   }
   
   
   ///////////////////////////////////////////////////////////////
   ///////////////////////////////////////////////////////////////
@@ -334,6 +360,11 @@ public class FilterFileSystem extends FileSystem {
   public void setVerifyChecksum(boolean verifyChecksum) {
   public void setVerifyChecksum(boolean verifyChecksum) {
     fs.setVerifyChecksum(verifyChecksum);
     fs.setVerifyChecksum(verifyChecksum);
   }
   }
+  
+  @Override
+  public void setWriteChecksum(boolean writeChecksum) {
+    fs.setVerifyChecksum(writeChecksum);
+  }
 
 
   @Override
   @Override
   public Configuration getConf() {
   public Configuration getConf() {

+ 9 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -24,6 +24,7 @@ import java.util.*;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 
 
 /****************************************************************
 /****************************************************************
  * Implement the FileSystem API for the checksumed local filesystem.
  * Implement the FileSystem API for the checksumed local filesystem.
@@ -34,21 +35,26 @@ import org.apache.hadoop.classification.InterfaceStability;
 public class LocalFileSystem extends ChecksumFileSystem {
 public class LocalFileSystem extends ChecksumFileSystem {
   static final URI NAME = URI.create("file:///");
   static final URI NAME = URI.create("file:///");
   static private Random rand = new Random();
   static private Random rand = new Random();
-  FileSystem rfs;
   
   
   public LocalFileSystem() {
   public LocalFileSystem() {
     this(new RawLocalFileSystem());
     this(new RawLocalFileSystem());
   }
   }
   
   
   public FileSystem getRaw() {
   public FileSystem getRaw() {
-    return rfs;
+    return getRawFileSystem();
   }
   }
     
     
   public LocalFileSystem(FileSystem rawLocalFileSystem) {
   public LocalFileSystem(FileSystem rawLocalFileSystem) {
     super(rawLocalFileSystem);
     super(rawLocalFileSystem);
-    rfs = rawLocalFileSystem;
   }
   }
     
     
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    // ctor didn't initialize the filtered fs
+    getRawFileSystem().initialize(uri, conf);
+  }
+  
   /** Convert a path to a File. */
   /** Convert a path to a File. */
   public File pathToFile(Path path) {
   public File pathToFile(Path path) {
     return ((RawLocalFileSystem)fs).pathToFile(path);
     return ((RawLocalFileSystem)fs).pathToFile(path);

+ 13 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java

@@ -41,7 +41,9 @@ import org.apache.hadoop.io.IOUtils;
  */
  */
 abstract class CommandWithDestination extends FsCommand {  
 abstract class CommandWithDestination extends FsCommand {  
   protected PathData dst;
   protected PathData dst;
-  protected boolean overwrite = false;
+  private boolean overwrite = false;
+  private boolean verifyChecksum = true;
+  private boolean writeChecksum = true;
   
   
   /**
   /**
    * 
    * 
@@ -53,6 +55,14 @@ abstract class CommandWithDestination extends FsCommand {
     overwrite = flag;
     overwrite = flag;
   }
   }
   
   
+  protected void setVerifyChecksum(boolean flag) {
+    verifyChecksum = flag;
+  }
+  
+  protected void setWriteChecksum(boolean flag) {
+    writeChecksum = flag;
+  }
+  
   /**
   /**
    *  The last arg is expected to be a local path, if only one argument is
    *  The last arg is expected to be a local path, if only one argument is
    *  given then the destination will be the current directory 
    *  given then the destination will be the current directory 
@@ -201,6 +211,7 @@ abstract class CommandWithDestination extends FsCommand {
    * @throws IOException if copy fails
    * @throws IOException if copy fails
    */ 
    */ 
   protected void copyFileToTarget(PathData src, PathData target) throws IOException {
   protected void copyFileToTarget(PathData src, PathData target) throws IOException {
+    src.fs.setVerifyChecksum(verifyChecksum);
     copyStreamToTarget(src.fs.open(src.path), target);
     copyStreamToTarget(src.fs.open(src.path), target);
   }
   }
   
   
@@ -217,6 +228,7 @@ abstract class CommandWithDestination extends FsCommand {
     if (target.exists && (target.stat.isDirectory() || !overwrite)) {
     if (target.exists && (target.stat.isDirectory() || !overwrite)) {
       throw new PathExistsException(target.toString());
       throw new PathExistsException(target.toString());
     }
     }
+    target.fs.setWriteChecksum(writeChecksum);
     PathData tempFile = null;
     PathData tempFile = null;
     try {
     try {
       tempFile = target.createTempFile(target+"._COPYING_");
       tempFile = target.createTempFile(target+"._COPYING_");

+ 2 - 29
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -25,7 +25,6 @@ import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.ChecksumFileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 
 
 /** Various commands for copy files */
 /** Various commands for copy files */
@@ -103,43 +102,17 @@ class CopyCommands {
       "to the local name.  <src> is kept.  When copying multiple,\n" +
       "to the local name.  <src> is kept.  When copying multiple,\n" +
       "files, the destination must be a directory.";
       "files, the destination must be a directory.";
 
 
-    /**
-     * The prefix for the tmp file used in copyToLocal.
-     * It must be at least three characters long, required by
-     * {@link java.io.File#createTempFile(String, String, File)}.
-     */
-    private boolean copyCrc;
-    private boolean verifyChecksum;
-
     @Override
     @Override
     protected void processOptions(LinkedList<String> args)
     protected void processOptions(LinkedList<String> args)
     throws IOException {
     throws IOException {
       CommandFormat cf = new CommandFormat(
       CommandFormat cf = new CommandFormat(
           1, Integer.MAX_VALUE, "crc", "ignoreCrc");
           1, Integer.MAX_VALUE, "crc", "ignoreCrc");
       cf.parse(args);
       cf.parse(args);
-      copyCrc = cf.getOpt("crc");
-      verifyChecksum = !cf.getOpt("ignoreCrc");
-      
+      setWriteChecksum(cf.getOpt("crc"));
+      setVerifyChecksum(!cf.getOpt("ignoreCrc"));
       setRecursive(true);
       setRecursive(true);
       getLocalDestination(args);
       getLocalDestination(args);
     }
     }
-
-    @Override
-    protected void copyFileToTarget(PathData src, PathData target)
-    throws IOException {
-      src.fs.setVerifyChecksum(verifyChecksum);
-
-      if (copyCrc && !(src.fs instanceof ChecksumFileSystem)) {
-        displayWarning(src.fs + ": Does not support checksums");
-        copyCrc = false;
-      }      
-
-      super.copyFileToTarget(src, target);
-      if (copyCrc) {
-        // should we delete real file if crc copy fails?
-        super.copyFileToTarget(src.getChecksumFile(), target.getChecksumFile());
-      }
-    }
   }
   }
 
 
   /**
   /**

+ 0 - 14
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java

@@ -27,7 +27,6 @@ import java.net.URISyntaxException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumFileSystem;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -169,19 +168,6 @@ public class PathData {
     }
     }
   }
   }
   
   
-  /**
-   * Return the corresponding crc data for a file.  Avoids exposing the fs
-   * contortions to the caller.  
-   * @return PathData of the crc file
-   * @throws IOException is anything goes wrong
-   */
-  public PathData getChecksumFile() throws IOException {
-    checkIfExists(FileTypeRequirement.SHOULD_NOT_BE_DIRECTORY);
-    ChecksumFileSystem srcFs = (ChecksumFileSystem)fs;
-    Path srcPath = srcFs.getChecksumFile(path);
-    return new PathData(srcFs.getRawFileSystem(), srcPath.toString());
-  }
-
   /**
   /**
    * Returns a temporary file for this PathData with the given extension.
    * Returns a temporary file for this PathData with the given extension.
    * The file will be deleted on exit.
    * The file will be deleted on exit.

+ 30 - 67
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java

@@ -19,9 +19,6 @@ package org.apache.hadoop.fs.viewfs;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.List;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -31,11 +28,11 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 
 
 /**
 /**
@@ -60,15 +57,14 @@ import org.apache.hadoop.util.Progressable;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
-class ChRootedFileSystem extends FileSystem {
-  private final FileSystem myFs; // the base file system whose root is changed
+class ChRootedFileSystem extends FilterFileSystem {
   private final URI myUri; // the base URI + the chRoot
   private final URI myUri; // the base URI + the chRoot
   private final Path chRootPathPart; // the root below the root of the base
   private final Path chRootPathPart; // the root below the root of the base
   private final String chRootPathPartString;
   private final String chRootPathPartString;
   private Path workingDir;
   private Path workingDir;
   
   
   protected FileSystem getMyFs() {
   protected FileSystem getMyFs() {
-    return myFs;
+    return getRawFileSystem();
   }
   }
   
   
   /**
   /**
@@ -84,37 +80,16 @@ class ChRootedFileSystem extends FileSystem {
   
   
   /**
   /**
    * Constructor
    * Constructor
-   * @param fs base file system
-   * @param theRoot chRoot for this file system
-   * @throws URISyntaxException
+   * @param uri base file system
+   * @param conf configuration
+   * @throws IOException 
    */
    */
-  public ChRootedFileSystem(final FileSystem fs, final Path theRoot)
-    throws URISyntaxException {
-    myFs = fs;
-    myFs.makeQualified(theRoot); //check that root is a valid path for fs
-                            // Would like to call myFs.checkPath(theRoot); 
-                            // but not public
-    chRootPathPart = new Path(theRoot.toUri().getPath());
+  public ChRootedFileSystem(final URI uri, Configuration conf)
+      throws IOException {
+    super(FileSystem.get(uri, conf));
+    chRootPathPart = new Path(uri.getPath());
     chRootPathPartString = chRootPathPart.toUri().getPath();
     chRootPathPartString = chRootPathPart.toUri().getPath();
-    try {
-      initialize(fs.getUri(), fs.getConf());
-    } catch (IOException e) { // This exception should not be thrown
-      throw new RuntimeException("This should not occur");
-    }
-    
-    /*
-     * We are making URI include the chrootedPath: e.g. file:///chrootedPath.
-     * This is questionable since Path#makeQualified(uri, path) ignores
-     * the pathPart of a uri. Since this class is internal we can ignore
-     * this issue but if we were to make it external then this needs
-     * to be resolved.
-     */
-    // Handle the two cases:
-    //              scheme:/// and scheme://authority/
-    myUri = new URI(myFs.getUri().toString() + 
-        (myFs.getUri().getAuthority() == null ? "" :  Path.SEPARATOR) +
-          chRootPathPart.toString().substring(1));
-
+    myUri = uri;
     workingDir = getHomeDirectory();
     workingDir = getHomeDirectory();
     // We don't use the wd of the myFs
     // We don't use the wd of the myFs
   }
   }
@@ -127,7 +102,6 @@ class ChRootedFileSystem extends FileSystem {
    */
    */
   public void initialize(final URI name, final Configuration conf)
   public void initialize(final URI name, final Configuration conf)
       throws IOException {
       throws IOException {
-    myFs.initialize(name, conf);
     super.initialize(name, conf);
     super.initialize(name, conf);
     setConf(conf);
     setConf(conf);
   }
   }
@@ -137,12 +111,6 @@ class ChRootedFileSystem extends FileSystem {
     return myUri;
     return myUri;
   }
   }
   
   
-  @Override
-  public Path makeQualified(final Path path) {
-    return myFs.makeQualified(path);
-    // NOT myFs.makeQualified(fullPath(path));
-  }
- 
   /**
   /**
    * Strip out the root from the path.
    * Strip out the root from the path.
    * @param p - fully qualified path p
    * @param p - fully qualified path p
@@ -175,7 +143,7 @@ class ChRootedFileSystem extends FileSystem {
   
   
   public Path getResolvedQualifiedPath(final Path f)
   public Path getResolvedQualifiedPath(final Path f)
       throws FileNotFoundException {
       throws FileNotFoundException {
-    return myFs.makeQualified(
+    return makeQualified(
         new Path(chRootPathPartString + f.toUri().toString()));
         new Path(chRootPathPartString + f.toUri().toString()));
   }
   }
   
   
@@ -199,14 +167,14 @@ class ChRootedFileSystem extends FileSystem {
   public FSDataOutputStream create(final Path f, final FsPermission permission,
   public FSDataOutputStream create(final Path f, final FsPermission permission,
       final boolean overwrite, final int bufferSize, final short replication,
       final boolean overwrite, final int bufferSize, final short replication,
       final long blockSize, final Progressable progress) throws IOException {
       final long blockSize, final Progressable progress) throws IOException {
-    return myFs.create(fullPath(f), permission, overwrite, bufferSize,
+    return super.create(fullPath(f), permission, overwrite, bufferSize,
         replication, blockSize, progress);
         replication, blockSize, progress);
   }
   }
 
 
   @Override
   @Override
   public boolean delete(final Path f, final boolean recursive) 
   public boolean delete(final Path f, final boolean recursive) 
       throws IOException {
       throws IOException {
-    return myFs.delete(fullPath(f), recursive);
+    return super.delete(fullPath(f), recursive);
   }
   }
   
   
 
 
@@ -219,95 +187,90 @@ class ChRootedFileSystem extends FileSystem {
   @Override
   @Override
   public BlockLocation[] getFileBlockLocations(final FileStatus fs, final long start,
   public BlockLocation[] getFileBlockLocations(final FileStatus fs, final long start,
       final long len) throws IOException {
       final long len) throws IOException {
-    return myFs.getFileBlockLocations(
+    return super.getFileBlockLocations(
         new ViewFsFileStatus(fs, fullPath(fs.getPath())), start, len);
         new ViewFsFileStatus(fs, fullPath(fs.getPath())), start, len);
   }
   }
 
 
   @Override
   @Override
   public FileChecksum getFileChecksum(final Path f) 
   public FileChecksum getFileChecksum(final Path f) 
       throws IOException {
       throws IOException {
-    return myFs.getFileChecksum(fullPath(f));
+    return super.getFileChecksum(fullPath(f));
   }
   }
 
 
   @Override
   @Override
   public FileStatus getFileStatus(final Path f) 
   public FileStatus getFileStatus(final Path f) 
       throws IOException {
       throws IOException {
-    return myFs.getFileStatus(fullPath(f));
+    return super.getFileStatus(fullPath(f));
   }
   }
 
 
   @Override
   @Override
   public FsStatus getStatus(Path p) throws IOException {
   public FsStatus getStatus(Path p) throws IOException {
-    return myFs.getStatus(fullPath(p));
+    return super.getStatus(fullPath(p));
   }
   }
 
 
   @Override
   @Override
   public FsServerDefaults getServerDefaults() throws IOException {
   public FsServerDefaults getServerDefaults() throws IOException {
-    return myFs.getServerDefaults();
+    return super.getServerDefaults();
   }
   }
 
 
   @Override
   @Override
   public FileStatus[] listStatus(final Path f) 
   public FileStatus[] listStatus(final Path f) 
       throws IOException {
       throws IOException {
-    return myFs.listStatus(fullPath(f));
+    return super.listStatus(fullPath(f));
   }
   }
   
   
   @Override
   @Override
   public boolean mkdirs(final Path f, final FsPermission permission)
   public boolean mkdirs(final Path f, final FsPermission permission)
       throws IOException {
       throws IOException {
-    return myFs.mkdirs(fullPath(f), permission);
+    return super.mkdirs(fullPath(f), permission);
   }
   }
 
 
   @Override
   @Override
   public FSDataInputStream open(final Path f, final int bufferSize) 
   public FSDataInputStream open(final Path f, final int bufferSize) 
     throws IOException {
     throws IOException {
-    return myFs.open(fullPath(f), bufferSize);
+    return super.open(fullPath(f), bufferSize);
   }
   }
   
   
   @Override
   @Override
   public FSDataOutputStream append(final Path f, final int bufferSize,
   public FSDataOutputStream append(final Path f, final int bufferSize,
       final Progressable progress) throws IOException {
       final Progressable progress) throws IOException {
-    return myFs.append(fullPath(f), bufferSize, progress);
+    return super.append(fullPath(f), bufferSize, progress);
   }
   }
 
 
   @Override
   @Override
   public boolean rename(final Path src, final Path dst) throws IOException {
   public boolean rename(final Path src, final Path dst) throws IOException {
     // note fullPath will check that paths are relative to this FileSystem.
     // note fullPath will check that paths are relative to this FileSystem.
     // Hence both are in same file system and a rename is valid
     // Hence both are in same file system and a rename is valid
-    return myFs.rename(fullPath(src), fullPath(dst)); 
+    return super.rename(fullPath(src), fullPath(dst)); 
   }
   }
   
   
   @Override
   @Override
   public void setOwner(final Path f, final String username,
   public void setOwner(final Path f, final String username,
       final String groupname)
       final String groupname)
     throws IOException {
     throws IOException {
-    myFs.setOwner(fullPath(f), username, groupname);
+    super.setOwner(fullPath(f), username, groupname);
   }
   }
 
 
   @Override
   @Override
   public void setPermission(final Path f, final FsPermission permission)
   public void setPermission(final Path f, final FsPermission permission)
     throws IOException {
     throws IOException {
-    myFs.setPermission(fullPath(f), permission);
+    super.setPermission(fullPath(f), permission);
   }
   }
 
 
   @Override
   @Override
   public boolean setReplication(final Path f, final short replication)
   public boolean setReplication(final Path f, final short replication)
     throws IOException {
     throws IOException {
-    return myFs.setReplication(fullPath(f), replication);
+    return super.setReplication(fullPath(f), replication);
   }
   }
 
 
   @Override
   @Override
   public void setTimes(final Path f, final long mtime, final long atime) 
   public void setTimes(final Path f, final long mtime, final long atime) 
       throws IOException {
       throws IOException {
-    myFs.setTimes(fullPath(f), mtime, atime);
-  }
-
-  @Override
-  public void setVerifyChecksum(final boolean verifyChecksum)  {
-    myFs.setVerifyChecksum(verifyChecksum);
+    super.setTimes(fullPath(f), mtime, atime);
   }
   }
   
   
   @Override
   @Override
-  public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
-    return myFs.getDelegationTokens(renewer);
+  public Path resolvePath(final Path p) throws IOException {
+    return super.resolvePath(fullPath(p));
   }
   }
 }
 }

+ 15 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -168,8 +168,7 @@ public class ViewFileSystem extends FileSystem {
         protected
         protected
         FileSystem getTargetFileSystem(final URI uri)
         FileSystem getTargetFileSystem(final URI uri)
           throws URISyntaxException, IOException {
           throws URISyntaxException, IOException {
-            return new ChRootedFileSystem(FileSystem.get(uri, config), 
-                new Path(uri.getPath()));
+            return new ChRootedFileSystem(uri, config);
         }
         }
 
 
         @Override
         @Override
@@ -464,10 +463,22 @@ public class ViewFileSystem extends FileSystem {
 
 
   @Override
   @Override
   public void setVerifyChecksum(final boolean verifyChecksum) { 
   public void setVerifyChecksum(final boolean verifyChecksum) { 
-    // This is a file system level operations, however ViewFileSystem 
-    // points to many file systems. Noop for ViewFileSystem.
+    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
+        fsState.getMountPoints();
+    for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
+      mount.target.targetFileSystem.setVerifyChecksum(verifyChecksum);
+    }
   }
   }
   
   
+  @Override
+  public void setWriteChecksum(final boolean writeChecksum) { 
+    List<InodeTree.MountPoint<FileSystem>> mountPoints = 
+        fsState.getMountPoints();
+    for (InodeTree.MountPoint<FileSystem> mount : mountPoints) {
+      mount.target.targetFileSystem.setWriteChecksum(writeChecksum);
+    }
+  }
+
   public MountPoint[] getMountPoints() {
   public MountPoint[] getMountPoints() {
     List<InodeTree.MountPoint<FileSystem>> mountPoints = 
     List<InodeTree.MountPoint<FileSystem>> mountPoints = 
                   fsState.getMountPoints();
                   fsState.getMountPoints();

+ 10 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java

@@ -34,6 +34,7 @@ import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 import javax.management.ObjectName;
 import javax.management.ReflectionException;
 import javax.management.ReflectionException;
+import javax.management.RuntimeMBeanException;
 import javax.management.openmbean.CompositeData;
 import javax.management.openmbean.CompositeData;
 import javax.management.openmbean.CompositeType;
 import javax.management.openmbean.CompositeType;
 import javax.management.openmbean.TabularData;
 import javax.management.openmbean.TabularData;
@@ -308,6 +309,15 @@ public class JMXJsonServlet extends HttpServlet {
     Object value = null;
     Object value = null;
     try {
     try {
       value = mBeanServer.getAttribute(oname, attName);
       value = mBeanServer.getAttribute(oname, attName);
+    } catch (RuntimeMBeanException e) {
+      // UnsupportedOperationExceptions happen in the normal course of business,
+      // so no need to log them as errors all the time.
+      if (e.getCause() instanceof UnsupportedOperationException) {
+        LOG.debug("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      } else {
+        LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      }
+      return;
     } catch (AttributeNotFoundException e) {
     } catch (AttributeNotFoundException e) {
       //Ignored the attribute was not found, which should never happen because the bean
       //Ignored the attribute was not found, which should never happen because the bean
       //just told us that it has this attribute, but if this happens just don't output
       //just told us that it has this attribute, but if this happens just don't output

+ 22 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -230,14 +230,34 @@ public class Credentials implements Writable {
  
  
   /**
   /**
    * Copy all of the credentials from one credential object into another.
    * Copy all of the credentials from one credential object into another.
+   * Existing secrets and tokens are overwritten.
    * @param other the credentials to copy
    * @param other the credentials to copy
    */
    */
   public void addAll(Credentials other) {
   public void addAll(Credentials other) {
+    addAll(other, true);
+  }
+
+  /**
+   * Copy all of the credentials from one credential object into another.
+   * Existing secrets and tokens are not overwritten.
+   * @param other the credentials to copy
+   */
+  public void mergeAll(Credentials other) {
+    addAll(other, false);
+  }
+
+  private void addAll(Credentials other, boolean overwrite) {
     for(Map.Entry<Text, byte[]> secret: other.secretKeysMap.entrySet()) {
     for(Map.Entry<Text, byte[]> secret: other.secretKeysMap.entrySet()) {
-      secretKeysMap.put(secret.getKey(), secret.getValue());
+      Text key = secret.getKey();
+      if (!secretKeysMap.containsKey(key) || overwrite) {
+        secretKeysMap.put(key, secret.getValue());
+      }
     }
     }
     for(Map.Entry<Text, Token<?>> token: other.tokenMap.entrySet()){
     for(Map.Entry<Text, Token<?>> token: other.tokenMap.entrySet()){
-      tokenMap.put(token.getKey(), token.getValue());
+      Text key = token.getKey();
+      if (!tokenMap.containsKey(key) || overwrite) {
+        tokenMap.put(key, token.getValue());
+      }
     }
     }
   }
   }
 }
 }

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java

@@ -72,14 +72,15 @@ public final class FileSystemTestHelper {
 
 
   public static String getAbsoluteTestRootDir(FileSystem fSys)
   public static String getAbsoluteTestRootDir(FileSystem fSys)
       throws IOException {
       throws IOException {
-    if (absTestRootDir == null) {
+    // NOTE: can't cache because of different filesystems!
+    //if (absTestRootDir == null) 
       if (TEST_ROOT_DIR.startsWith("/")) {
       if (TEST_ROOT_DIR.startsWith("/")) {
         absTestRootDir = TEST_ROOT_DIR;
         absTestRootDir = TEST_ROOT_DIR;
       } else {
       } else {
         absTestRootDir = fSys.getWorkingDirectory().toString() + "/"
         absTestRootDir = fSys.getWorkingDirectory().toString() + "/"
             + TEST_ROOT_DIR;
             + TEST_ROOT_DIR;
       }
       }
-    }
+    //}
     return absTestRootDir;
     return absTestRootDir;
   }
   }
   
   

+ 54 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java

@@ -203,4 +203,58 @@ public class TestChecksumFileSystem {
     String str = readFile(localFs, testPath, 1024);
     String str = readFile(localFs, testPath, 1024);
     assertEquals("testing stale checksum", str);
     assertEquals("testing stale checksum", str);
   }
   }
+  
+  @Test
+  public void testRenameFileToFile() throws Exception {
+    Path srcPath = new Path(TEST_ROOT_DIR, "testRenameSrc");
+    Path dstPath = new Path(TEST_ROOT_DIR, "testRenameDst");
+    verifyRename(srcPath, dstPath, false);
+  }
+
+  @Test
+  public void testRenameFileIntoDir() throws Exception {
+    Path srcPath = new Path(TEST_ROOT_DIR, "testRenameSrc");
+    Path dstPath = new Path(TEST_ROOT_DIR, "testRenameDir");
+    localFs.mkdirs(dstPath);
+    verifyRename(srcPath, dstPath, true);
+  }
+
+  @Test
+  public void testRenameFileIntoDirFile() throws Exception {
+    Path srcPath = new Path(TEST_ROOT_DIR, "testRenameSrc");
+    Path dstPath = new Path(TEST_ROOT_DIR, "testRenameDir/testRenameDst");
+    assertTrue(localFs.mkdirs(dstPath));
+    verifyRename(srcPath, dstPath, false);
+  }
+
+
+  void verifyRename(Path srcPath, Path dstPath, boolean dstIsDir)
+      throws Exception { 
+    localFs.delete(srcPath,true);
+    localFs.delete(dstPath,true);
+    
+    Path realDstPath = dstPath;
+    if (dstIsDir) {
+      localFs.mkdirs(dstPath);
+      realDstPath = new Path(dstPath, srcPath.getName());
+    }
+    
+    // ensure file + checksum are moved
+    writeFile(localFs, srcPath, 1);
+    assertTrue(localFs.exists(localFs.getChecksumFile(srcPath)));
+    assertTrue(localFs.rename(srcPath, dstPath));
+    assertTrue(localFs.exists(localFs.getChecksumFile(realDstPath)));
+
+    // create a file with no checksum, rename, ensure dst checksum is removed    
+    writeFile(localFs.getRawFileSystem(), srcPath, 1);
+    assertFalse(localFs.exists(localFs.getChecksumFile(srcPath)));
+    assertTrue(localFs.rename(srcPath, dstPath));
+    assertFalse(localFs.exists(localFs.getChecksumFile(realDstPath)));
+    
+    // create file with checksum, rename over prior dst with no checksum
+    writeFile(localFs, srcPath, 1);
+    assertTrue(localFs.exists(localFs.getChecksumFile(srcPath)));
+    assertTrue(localFs.rename(srcPath, dstPath));
+    assertTrue(localFs.exists(localFs.getChecksumFile(realDstPath)));
+  }
 }
 }

+ 97 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellCopy.java

@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFsShellCopy {  
+  static Configuration conf;
+  static FsShell shell; 
+  static LocalFileSystem lfs;
+  static Path testRootDir, srcPath, dstPath;
+  
+  @BeforeClass
+  public static void setup() throws Exception {
+    conf = new Configuration();
+    shell = new FsShell(conf);
+    lfs = FileSystem.getLocal(conf);
+    testRootDir = new Path(
+        System.getProperty("test.build.data","test/build/data"), "testShellCopy");
+    lfs.mkdirs(testRootDir);    
+    srcPath = new Path(testRootDir, "srcFile");
+    dstPath = new Path(testRootDir, "dstFile");
+  }
+  
+  @Before
+  public void prepFiles() throws Exception {
+    lfs.setVerifyChecksum(true);
+    lfs.setWriteChecksum(true);
+    
+    lfs.delete(srcPath, true);
+    lfs.delete(dstPath, true);
+    FSDataOutputStream out = lfs.create(srcPath);
+    out.writeChars("hi");
+    out.close();
+    assertTrue(lfs.exists(lfs.getChecksumFile(srcPath)));
+  }
+
+  @Test
+  public void testCopyNoCrc() throws Exception {
+    shellRun(0, "-get", srcPath.toString(), dstPath.toString());
+    checkPath(dstPath, false);
+  }
+
+  @Test
+  public void testCopyCrc() throws Exception {
+    shellRun(0, "-get", "-crc", srcPath.toString(), dstPath.toString());
+    checkPath(dstPath, true);
+  }
+
+  
+  @Test
+  public void testCorruptedCopyCrc() throws Exception {
+    FSDataOutputStream out = lfs.getRawFileSystem().create(srcPath);
+    out.writeChars("bang");
+    out.close();
+    shellRun(1, "-get", srcPath.toString(), dstPath.toString());
+  }
+
+  @Test
+  public void testCorruptedCopyIgnoreCrc() throws Exception {
+    shellRun(0, "-get", "-ignoreCrc", srcPath.toString(), dstPath.toString());
+    checkPath(dstPath, false);
+  }
+
+  private void checkPath(Path p, boolean expectChecksum) throws IOException {
+    assertTrue(lfs.exists(p));
+    boolean hasChecksum = lfs.exists(lfs.getChecksumFile(p));
+    assertEquals(expectChecksum, hasChecksum);
+  }
+
+  private void shellRun(int n, String ... args) throws Exception {
+    assertEquals(n, shell.run(args));
+  }
+}

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java

@@ -51,7 +51,7 @@ public class TestChRootedFileSystem {
 
 
 
 
     // ChRoot to the root of the testDirectory
     // ChRoot to the root of the testDirectory
-    fSys = new ChRootedFileSystem(fSysTarget, chrootedTo);
+    fSys = new ChRootedFileSystem(chrootedTo.toUri(), conf);
   }
   }
 
 
   @After
   @After

+ 102 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemDelegation.java

@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.viewfs;
+
+import java.io.IOException;
+import java.net.URI;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.*;
+import static org.junit.Assert.*;
+
+/**
+ * Verify that viewfs propagates certain methods to the underlying fs 
+ */
+public class TestViewFileSystemDelegation { //extends ViewFileSystemTestSetup {
+  static Configuration conf;
+  static FileSystem viewFs;
+  static FakeFileSystem fs1;
+  static FakeFileSystem fs2;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    conf = ViewFileSystemTestSetup.configWithViewfsScheme();    
+    fs1 = setupFileSystem(new URI("fs1:/"), FakeFileSystem.class);
+    fs2 = setupFileSystem(new URI("fs2:/"), FakeFileSystem.class);
+    viewFs = FileSystem.get(FsConstants.VIEWFS_URI, conf);
+  }
+  
+  static FakeFileSystem setupFileSystem(URI uri, Class clazz)
+      throws Exception {
+    String scheme = uri.getScheme();
+    conf.set("fs."+scheme+".impl", clazz.getName());
+    FakeFileSystem fs = (FakeFileSystem)FileSystem.get(uri, conf);
+    assertEquals(uri, fs.getUri());
+    Path targetPath = FileSystemTestHelper.getAbsoluteTestRootPath(fs);
+    ConfigUtil.addLink(conf, "/mounts/"+scheme, targetPath.toUri());
+    return fs;
+  }
+
+  @Test
+  public void testSanity() {
+    assertEquals("fs1:/", fs1.getUri().toString());
+    assertEquals("fs2:/", fs2.getUri().toString());
+  }
+  
+  @Test
+  public void testVerifyChecksum() throws Exception {
+    checkVerifyChecksum(false);
+    checkVerifyChecksum(true);
+  }
+
+  void checkVerifyChecksum(boolean flag) {
+    viewFs.setVerifyChecksum(flag);
+    assertEquals(flag, fs1.getVerifyChecksum());
+    assertEquals(flag, fs2.getVerifyChecksum());
+  }
+
+  static class FakeFileSystem extends LocalFileSystem {
+    boolean verifyChecksum = true;
+    URI uri;
+    
+    @Override
+    public void initialize(URI uri, Configuration conf) throws IOException {
+      super.initialize(uri, conf);
+      this.uri = uri;
+    }
+    
+    @Override
+    public URI getUri() {
+      return uri;
+    }
+    
+    @Override
+    public void setVerifyChecksum(boolean verifyChecksum) {
+      this.verifyChecksum = verifyChecksum;
+    }
+    
+    public boolean getVerifyChecksum(){
+      return verifyChecksum;
+    }
+  }
+}

+ 77 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestCredentials.java

@@ -137,4 +137,81 @@ public class TestCredentials {
     }
     }
     tmpFileName.delete();
     tmpFileName.delete();
   }
   }
+
+  static Text secret[] = {
+      new Text("secret1"),
+      new Text("secret2"),
+      new Text("secret3"),
+      new Text("secret4")
+  };
+  static Text service[] = {
+      new Text("service1"),
+      new Text("service2"),
+      new Text("service3"),
+      new Text("service4")
+  };
+  static Token<?> token[] = {
+      new Token<TokenIdentifier>(),
+      new Token<TokenIdentifier>(),
+      new Token<TokenIdentifier>(),
+      new Token<TokenIdentifier>()
+  };
+  
+  @Test
+  public void addAll() {
+    Credentials creds = new Credentials();
+    creds.addToken(service[0], token[0]);
+    creds.addToken(service[1], token[1]);
+    creds.addSecretKey(secret[0], secret[0].getBytes());
+    creds.addSecretKey(secret[1], secret[1].getBytes());
+
+    Credentials credsToAdd = new Credentials();
+    // one duplicate with different value, one new
+    credsToAdd.addToken(service[0], token[3]);
+    credsToAdd.addToken(service[2], token[2]);
+    credsToAdd.addSecretKey(secret[0], secret[3].getBytes());
+    credsToAdd.addSecretKey(secret[2], secret[2].getBytes());
+    
+    creds.addAll(credsToAdd);
+    assertEquals(3, creds.numberOfTokens());
+    assertEquals(3, creds.numberOfSecretKeys());
+    // existing token & secret should be overwritten
+    assertEquals(token[3], creds.getToken(service[0]));
+    assertEquals(secret[3], new Text(creds.getSecretKey(secret[0])));
+    // non-duplicate token & secret should be present
+    assertEquals(token[1], creds.getToken(service[1]));
+    assertEquals(secret[1], new Text(creds.getSecretKey(secret[1])));
+    // new token & secret should be added
+    assertEquals(token[2], creds.getToken(service[2]));
+    assertEquals(secret[2], new Text(creds.getSecretKey(secret[2])));
+  }
+
+  @Test
+  public void mergeAll() {
+    Credentials creds = new Credentials();
+    creds.addToken(service[0], token[0]);
+    creds.addToken(service[1], token[1]);
+    creds.addSecretKey(secret[0], secret[0].getBytes());
+    creds.addSecretKey(secret[1], secret[1].getBytes());
+    
+    Credentials credsToAdd = new Credentials();
+    // one duplicate with different value, one new
+    credsToAdd.addToken(service[0], token[3]);
+    credsToAdd.addToken(service[2], token[2]);
+    credsToAdd.addSecretKey(secret[0], secret[3].getBytes());
+    credsToAdd.addSecretKey(secret[2], secret[2].getBytes());
+    
+    creds.mergeAll(credsToAdd);
+    assertEquals(3, creds.numberOfTokens());
+    assertEquals(3, creds.numberOfSecretKeys());
+    // existing token & secret should not be overwritten
+    assertEquals(token[0], creds.getToken(service[0]));
+    assertEquals(secret[0], new Text(creds.getSecretKey(secret[0])));
+    // non-duplicate token & secret should be present
+    assertEquals(token[1], creds.getToken(service[1]));
+    assertEquals(secret[1], new Text(creds.getSecretKey(secret[1])));
+    // new token & secret should be added
+    assertEquals(token[2], creds.getToken(service[2]));
+    assertEquals(secret[2], new Text(creds.getSecretKey(secret[2])));
  }
  }
+}

+ 117 - 31
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -45,10 +45,16 @@ Trunk (unreleased changes)
     HDFS-2697. Move RefreshAuthPolicy, RefreshUserMappings, GetUserMappings 
     HDFS-2697. Move RefreshAuthPolicy, RefreshUserMappings, GetUserMappings 
     protocol to protocol buffers. (jitendra)
     protocol to protocol buffers. (jitendra)
 
 
+    HDFS-2880. Protobuf changes in DatanodeProtocol to add multiple storages.
+    (suresh)
+
+    HDFS-2899. Service protocol changes in DatanodeProtocol to add multiple 
+    storages. (suresh)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HADOOP-7524 Change RPC to allow multiple protocols including multuple 
     HADOOP-7524 Change RPC to allow multiple protocols including multuple 
-    versions of the same protocol (sanjay Radia)
+    versions of the same protocol (Sanjay Radia)
 
 
     HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
     HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
                HdfsConstants. (Harsh J Chouraria via atm)
                HdfsConstants. (Harsh J Chouraria via atm)
@@ -83,12 +89,16 @@ Trunk (unreleased changes)
 
 
     HDFS-2334. Add Closeable to JournalManager. (Ivan Kelly via jitendra)
     HDFS-2334. Add Closeable to JournalManager. (Ivan Kelly via jitendra)
 
 
+    HDFS-2572. Remove unnecessary double-check in DN#getHostName. (harsh)
+
     HDFS-2564. Cleanup unnecessary exceptions thrown and unnecessary casts.
     HDFS-2564. Cleanup unnecessary exceptions thrown and unnecessary casts.
     (Hari Mankude via eli)
     (Hari Mankude via eli)
 
 
     HDFS-2410. Further cleanup of hardcoded configuration keys and values.
     HDFS-2410. Further cleanup of hardcoded configuration keys and values.
     (suresh)
     (suresh)
 
 
+    HDFS-2857. Cleanup BlockInfo class. (suresh)
+
     HADOOP-7862   Hdfs changes to work with HADOOP 7862: 
     HADOOP-7862   Hdfs changes to work with HADOOP 7862: 
     Move the support for multiple protocols to lower layer so that Writable,
     Move the support for multiple protocols to lower layer so that Writable,
     PB and Avro can all use it (Sanjay)
     PB and Avro can all use it (Sanjay)
@@ -100,9 +110,21 @@ Trunk (unreleased changes)
 
 
     HDFS-2651 ClientNameNodeProtocol Translators for Protocol Buffers (sanjay)
     HDFS-2651 ClientNameNodeProtocol Translators for Protocol Buffers (sanjay)
 
 
-    HDFS-2650. Replace @inheritDoc with @Override. (Hari Mankude via suresh).
+    HDFS-2650. Replace @inheritDoc with @Override. (Hari Mankude via suresh)
+
+    HDFS-2669. Enable protobuf rpc for ClientNamenodeProtocol. (Sanjay Radia)
+
+    HDFS-2801. Provide a method in client side translators to check for a 
+    methods supported in underlying protocol. (jitendra)
 
 
-    HDFS-2669 Enable protobuf rpc for ClientNamenodeProtocol
+    HDFS-208. name node should warn if only one dir is listed in dfs.name.dir.
+    (Uma Maheswara Rao G via eli)
+
+    HDS-2895. Remove Writable wire protocol types and translators to
+    complete transition to protocol buffers. (suresh)
+
+    HDFS-2786. Fix host-based token incompatibilities in DFSUtil. (Kihwal Lee
+    via jitendra)
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS
     HDFS-2477. Optimize computing the diff between a block report and the
     HDFS-2477. Optimize computing the diff between a block report and the
@@ -175,7 +197,31 @@ Trunk (unreleased changes)
     HDFS-2759. Pre-allocate HDFS edit log files after writing version number.
     HDFS-2759. Pre-allocate HDFS edit log files after writing version number.
     (atm)
     (atm)
 
 
-Release 0.23.1 - UNRELEASED
+    HDFS-2908. Add apache license header for StorageReport.java. (Brandon Li
+    via jitendra)
+
+Release 0.23.2 - UNRELEASED 
+
+  INCOMPATIBLE CHANGES
+
+    HDFS-2887. FSVolume, is a part of FSDatasetInterface implementation, should
+    not be referred outside FSDataset.  A new FSVolumeInterface is defined.
+    The BlockVolumeChoosingPolicy.chooseVolume(..) method signature is also
+    updated.  (szetszwo)
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+    HDFS-2923. Namenode IPC handler count uses the wrong configuration key
+    (todd)
+
+    HDFS-2764. TestBackupNode is racy. (atm)
+
+Release 0.23.1 - 2012-02-08 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
@@ -206,9 +252,6 @@ Release 0.23.1 - UNRELEASED
 
 
     HDFS-2563. Some cleanup in BPOfferService. (todd)
     HDFS-2563. Some cleanup in BPOfferService. (todd)
 
 
-    HDFS-208. name node should warn if only one dir is listed in dfs.name.dir.
-    (Uma Maheswara Rao G via eli)
-
     HDFS-2568. Use a set to manage child sockets in XceiverServer.
     HDFS-2568. Use a set to manage child sockets in XceiverServer.
     (harsh via eli)
     (harsh via eli)
 
 
@@ -229,6 +272,8 @@ Release 0.23.1 - UNRELEASED
     HDFS-2604. Add a log message to show if WebHDFS is enabled and a
     HDFS-2604. Add a log message to show if WebHDFS is enabled and a
     configuration section in the forrest doc.  (szetszwo)
     configuration section in the forrest doc.  (szetszwo)
 
 
+    HDFS-2511. Add dev script to generate HDFS protobufs. (tucu)
+
     HDFS-2654. Make BlockReaderLocal not extend RemoteBlockReader2. (eli)
     HDFS-2654. Make BlockReaderLocal not extend RemoteBlockReader2. (eli)
 
 
     HDFS-2675. Reduce warning verbosity when double-closing edit logs
     HDFS-2675. Reduce warning verbosity when double-closing edit logs
@@ -244,29 +289,23 @@ Release 0.23.1 - UNRELEASED
 
 
     HDFS-2710. Add HDFS tests related to HADOOP-7933. (sid via suresh)
     HDFS-2710. Add HDFS tests related to HADOOP-7933. (sid via suresh)
 
 
-    HDFS-2294. Download of commons-daemon TAR should not be under target (tucu)
-
-    HDFS-2322. the build fails in Windows because commons-daemon TAR cannot be 
-    fetched. (tucu)
-
-    HDFS-2511. Add dev script to generate HDFS protobufs. (tucu)
-
     HDFS-2349. Corruption detected during block transfers between DNs
     HDFS-2349. Corruption detected during block transfers between DNs
     should log a WARN instead of INFO. (harsh)
     should log a WARN instead of INFO. (harsh)
 
 
-    HDFS-2572. Remove unnecessary double-check in DN#getHostName. (harsh)
-
-    HDFS-2729. Update BlockManager's comments regarding the invalid block set (harsh)
+    HDFS-2729. Update BlockManager's comments regarding the invalid block
+    set (harsh)
 
 
-    HDFS-2726. Fix a logging issue under DFSClient's createBlockOutputStream method (harsh)
+    HDFS-2726. Fix a logging issue under DFSClient's createBlockOutputStream
+    method (harsh)
 
 
     HDFS-554. Use System.arraycopy in BlockInfo.ensureCapacity. (harsh)
     HDFS-554. Use System.arraycopy in BlockInfo.ensureCapacity. (harsh)
 
 
-    HDFS-1314. Make dfs.blocksize accept size-indicating prefixes (Sho Shimauchi via harsh)
+    HDFS-1314. Make dfs.blocksize accept size-indicating prefixes.
+    (Sho Shimauchi via harsh)
 
 
     HDFS-69. Improve the 'dfsadmin' commandline help. (harsh)
     HDFS-69. Improve the 'dfsadmin' commandline help. (harsh)
 
 
-    HDFS-2788. HdfsServerConstants#DN_KEEPALIVE_TIMEOUT is dead code (eli)
+    HDFS-2788. HdfsServerConstants#DN_KEEPALIVE_TIMEOUT is dead code. (eli)
 
 
     HDFS-362.  FSEditLog should not writes long and short as UTF8, and should
     HDFS-362.  FSEditLog should not writes long and short as UTF8, and should
     not use ArrayWritable for writing non-array items.  (Uma Maheswara Rao G
     not use ArrayWritable for writing non-array items.  (Uma Maheswara Rao G
@@ -277,9 +316,21 @@ Release 0.23.1 - UNRELEASED
 
 
     HDFS-2817. Combine the two TestSafeMode test suites. (todd)
     HDFS-2817. Combine the two TestSafeMode test suites. (todd)
 
 
-    HDFS-2818. Fix a missing space issue in HDFS webapps' title tags. (Devaraj K via harsh)
+    HDFS-2818. Fix a missing space issue in HDFS webapps' title tags.
+    (Devaraj K via harsh)
+
+    HDFS-2397. Undeprecate SecondaryNameNode. (eli)
+
+    HDFS-2814 NamenodeMXBean does not account for svn revision in the version 
+    information. (Hitesh Shah via jitendra)
+
+    HDFS-2784. Update hftp and hdfs for host-based token support. 
+    (Kihwal Lee via jitendra)
+
+    HDFS-2785. Update webhdfs and httpfs for host-based token support.
+    (Robert Joseph Evans via jitendra)
 
 
-    HDFS-2397. Undeprecate SecondaryNameNode (eli)
+    HDFS-2868. Expose xceiver counts via the DataNode MXBean. (harsh)
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
@@ -301,6 +352,11 @@ Release 0.23.1 - UNRELEASED
     HDFS-2826. Add test case for HDFS-1476 (safemode can initialize
     HDFS-2826. Add test case for HDFS-1476 (safemode can initialize
     replication queues before exiting) (todd)
     replication queues before exiting) (todd)
 
 
+    HDFS-2864. Remove some redundant methods and the constant METADATA_VERSION
+    from FSDataset.  (szetszwo)
+
+    HDFS-2879. Change FSDataset to package private.  (szetszwo)
+
   BUG FIXES
   BUG FIXES
 
 
     HDFS-2541. For a sufficiently large value of blocks, the DN Scanner 
     HDFS-2541. For a sufficiently large value of blocks, the DN Scanner 
@@ -320,6 +376,11 @@ Release 0.23.1 - UNRELEASED
 
 
     HDFS-2596. TestDirectoryScanner doesn't test parallel scans. (eli)
     HDFS-2596. TestDirectoryScanner doesn't test parallel scans. (eli)
 
 
+    HDFS-2606. webhdfs client filesystem impl must set the content-type 
+    header for create/append. (tucu)
+
+    HDFS-2614. hadoop dist tarball is missing hdfs headers. (tucu)
+ 
     HDFS-2653. DFSClient should cache whether addrs are non-local when
     HDFS-2653. DFSClient should cache whether addrs are non-local when
     short-circuiting is enabled. (eli)
     short-circuiting is enabled. (eli)
 
 
@@ -331,20 +392,17 @@ Release 0.23.1 - UNRELEASED
     HDFS-2553. Fix BlockPoolSliceScanner spinning in a tight loop (Uma
     HDFS-2553. Fix BlockPoolSliceScanner spinning in a tight loop (Uma
     Maheswara Rao G via todd)
     Maheswara Rao G via todd)
 
 
+    HDFS-2658. HttpFS introduced 70 javadoc warnings. (tucu)
+
     HDFS-2706. Use configuration for blockInvalidateLimit if it is set.
     HDFS-2706. Use configuration for blockInvalidateLimit if it is set.
     (szetszwo)
     (szetszwo)
 
 
-    HDFS-2606. webhdfs client filesystem impl must set the content-type 
-    header for create/append. (tucu)
-
-    HDFS-2658. HttpFS introduced 70 javadoc warnings. (tucu)
-
     HDFS-2646. Hadoop HttpFS introduced 4 findbug warnings. (tucu)
     HDFS-2646. Hadoop HttpFS introduced 4 findbug warnings. (tucu)
 
 
     HDFS-2657. TestHttpFSServer and TestServerWebApp are failing on trunk. 
     HDFS-2657. TestHttpFSServer and TestServerWebApp are failing on trunk. 
     (tucu)
     (tucu)
 
 
-    HttpFS server should check that upload requests have correct 
+    HDFS-2705. HttpFS server should check that upload requests have correct 
     content-type. (tucu)
     content-type. (tucu)
 
 
     HDFS-2707. HttpFS should read the hadoop-auth secret from a file 
     HDFS-2707. HttpFS should read the hadoop-auth secret from a file 
@@ -366,15 +424,31 @@ Release 0.23.1 - UNRELEASED
 
 
     HDFS-442. dfsthroughput in test jar throws NPE (harsh)
     HDFS-442. dfsthroughput in test jar throws NPE (harsh)
 
 
-    HDFS-2836. HttpFSServer still has 2 javadoc warnings in trunk (revans2 via tucu)
+    HDFS-2836. HttpFSServer still has 2 javadoc warnings in trunk.
+    (revans2 via tucu)
 
 
-    HDFS-2837. mvn javadoc:javadoc not seeing LimitedPrivate class (revans2 via tucu)
+    HDFS-2837. mvn javadoc:javadoc not seeing LimitedPrivate class
+    (revans2 via tucu)
 
 
-    HDFS-2840. TestHostnameFilter should work with localhost or localhost.localdomain (tucu)
+    HDFS-2840. TestHostnameFilter should work with localhost or 
+    localhost.localdomain (tucu)
 
 
     HDFS-2791. If block report races with closing of file, replica is
     HDFS-2791. If block report races with closing of file, replica is
     incorrectly marked corrupt. (todd)
     incorrectly marked corrupt. (todd)
 
 
+    HDFS-2827.  When the parent of a directory is the root, renaming the
+    directory results in leases updated incorrectly.  (Uma Maheswara Rao G
+    via szetszwo)
+
+    HDFS-2835. Fix findbugs and javadoc issue with GetConf.java.
+    (suresh)
+
+    HDFS-2889. getNumCurrentReplicas is package private but should be public on
+    0.23 (see HDFS-2408). (Gregory Chanan via atm)
+
+    HDFS-2893. The start/stop scripts don't start/stop the 2NN when
+    using the default configuration. (eli)
+
 Release 0.23.0 - 2011-11-01 
 Release 0.23.0 - 2011-11-01 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -1126,6 +1200,11 @@ Release 0.23.0 - 2011-11-01
     HDFS-2493. Remove reference to FSNamesystem in blockmanagement classes.
     HDFS-2493. Remove reference to FSNamesystem in blockmanagement classes.
     (szetszwo)
     (szetszwo)
 
 
+    HDFS-2294. Download of commons-daemon TAR should not be under target (tucu)
+
+    HDFS-2322. the build fails in Windows because commons-daemon TAR cannot be 
+    fetched. (tucu)
+
     HDFS-2436. Change FSNamesystem.setTimes(..) for allowing setting times on
     HDFS-2436. Change FSNamesystem.setTimes(..) for allowing setting times on
     directories.  (Uma Maheswara Rao G via szetszwo)
     directories.  (Uma Maheswara Rao G via szetszwo)
 
 
@@ -1647,8 +1726,15 @@ Release 0.22.1 - Unreleased
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
+    HDFS-2718. Optimize OP_ADD in edits loading. (shv)
+
+    HDFS-2886. CreateEditLogs should generate a realistic edit log. (shv)
+
   BUG FIXES
   BUG FIXES
 
 
+    HDFS-2877. If locking of a storage dir fails, it will remove the other
+    NN's lock file on exit. (todd)
+
 Release 0.22.0 - 2011-11-29
 Release 0.22.0 - 2011-11-29
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 3 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh

@@ -59,7 +59,7 @@ echo "Starting namenodes on [$NAMENODES]"
   --script "$bin/hdfs" start namenode $nameStartOpt
   --script "$bin/hdfs" start namenode $nameStartOpt
 
 
 #---------------------------------------------------------
 #---------------------------------------------------------
-# datanodes (using defalut slaves file)
+# datanodes (using default slaves file)
 
 
 if [ -n "$HADOOP_SECURE_DN_USER" ]; then
 if [ -n "$HADOOP_SECURE_DN_USER" ]; then
   echo \
   echo \
@@ -74,22 +74,13 @@ fi
 #---------------------------------------------------------
 #---------------------------------------------------------
 # secondary namenodes (if any)
 # secondary namenodes (if any)
 
 
-# if there are no secondary namenodes configured it returns
-# 0.0.0.0 or empty string
 SECONDARY_NAMENODES=$($HADOOP_PREFIX/bin/hdfs getconf -secondarynamenodes 2>&-)
 SECONDARY_NAMENODES=$($HADOOP_PREFIX/bin/hdfs getconf -secondarynamenodes 2>&-)
-SECONDARY_NAMENODES=${SECONDARY_NAMENODES:='0.0.0.0'}
 
 
-if [ "$SECONDARY_NAMENODES" = '0.0.0.0' ] ; then
-  echo \
-    "Secondary namenodes are not configured. " \
-    "Cannot start secondary namenodes."
-else
-  echo "Starting secondary namenodes [$SECONDARY_NAMENODES]"
+echo "Starting secondary namenodes [$SECONDARY_NAMENODES]"
 
 
-  "$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \
+"$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \
     --config "$HADOOP_CONF_DIR" \
     --config "$HADOOP_CONF_DIR" \
     --hostnames "$SECONDARY_NAMENODES" \
     --hostnames "$SECONDARY_NAMENODES" \
     --script "$bin/hdfs" start secondarynamenode
     --script "$bin/hdfs" start secondarynamenode
-fi
 
 
 # eof
 # eof

+ 2 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh

@@ -50,22 +50,13 @@ fi
 #---------------------------------------------------------
 #---------------------------------------------------------
 # secondary namenodes (if any)
 # secondary namenodes (if any)
 
 
-# if there are no secondary namenodes configured it returns
-# 0.0.0.0 or empty string
 SECONDARY_NAMENODES=$($HADOOP_PREFIX/bin/hdfs getconf -secondarynamenodes 2>&-)
 SECONDARY_NAMENODES=$($HADOOP_PREFIX/bin/hdfs getconf -secondarynamenodes 2>&-)
-SECONDARY_NAMENODES=${SECONDARY_NAMENODES:-'0.0.0.0'}
 
 
-if [ "$SECONDARY_NAMENODES" = '0.0.0.0' ] ; then
-  echo \
-    "Secondary namenodes are not configured. " \
-    "Cannot stop secondary namenodes."
-else
-  echo "Stopping secondary namenodes [$SECONDARY_NAMENODES]"
+echo "Stopping secondary namenodes [$SECONDARY_NAMENODES]"
 
 
-  "$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \
+"$HADOOP_PREFIX/sbin/hadoop-daemons.sh" \
     --config "$HADOOP_CONF_DIR" \
     --config "$HADOOP_CONF_DIR" \
     --hostnames "$SECONDARY_NAMENODES" \
     --hostnames "$SECONDARY_NAMENODES" \
     --script "$bin/hdfs" stop secondarynamenode
     --script "$bin/hdfs" stop secondarynamenode
-fi
 
 
 # eof
 # eof

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/hdfs_user_guide.xml

@@ -253,9 +253,7 @@
      The secondary NameNode merges the fsimage and the edits log files periodically
      The secondary NameNode merges the fsimage and the edits log files periodically
      and keeps edits log size within a limit. It is usually run on a
      and keeps edits log size within a limit. It is usually run on a
      different machine than the primary NameNode since its memory requirements
      different machine than the primary NameNode since its memory requirements
-     are on the same order as the primary NameNode. The secondary
-     NameNode is started by <code>bin/start-dfs.sh</code> on the nodes 
-     specified in <code>conf/masters</code> file.
+     are on the same order as the primary NameNode.
    </p>
    </p>
    <p>
    <p>
      The start of the checkpoint process on the secondary NameNode is 
      The start of the checkpoint process on the secondary NameNode is 

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -31,13 +31,12 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset;
 import org.apache.hadoop.hdfs.util.DirectBufferPool;
 import org.apache.hadoop.hdfs.util.DirectBufferPool;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
@@ -183,7 +182,7 @@ class BlockReaderLocal implements BlockReader {
         BlockMetadataHeader header = BlockMetadataHeader
         BlockMetadataHeader header = BlockMetadataHeader
             .readHeader(new DataInputStream(checksumIn));
             .readHeader(new DataInputStream(checksumIn));
         short version = header.getVersion();
         short version = header.getVersion();
-        if (version != FSDataset.METADATA_VERSION) {
+        if (version != BlockMetadataHeader.VERSION) {
           LOG.warn("Wrong version (" + version + ") for metadata file for "
           LOG.warn("Wrong version (" + version + ") for metadata file for "
               + blk + " ignoring ...");
               + blk + " ignoring ...");
         }
         }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -663,7 +663,7 @@ public class DFSClient implements java.io.Closeable {
                DelegationTokenIdentifier.stringifyToken(delToken));
                DelegationTokenIdentifier.stringifyToken(delToken));
       ClientProtocol nn = 
       ClientProtocol nn = 
         DFSUtil.createNamenode
         DFSUtil.createNamenode
-           (NameNode.getAddress(token.getService().toString()),
+           (SecurityUtil.getTokenServiceAddr(delToken),
             conf, UserGroupInformation.getCurrentUser());
             conf, UserGroupInformation.getCurrentUser());
       try {
       try {
         return nn.renewDelegationToken(delToken);
         return nn.renewDelegationToken(delToken);
@@ -681,7 +681,7 @@ public class DFSClient implements java.io.Closeable {
       LOG.info("Cancelling " + 
       LOG.info("Cancelling " + 
                DelegationTokenIdentifier.stringifyToken(delToken));
                DelegationTokenIdentifier.stringifyToken(delToken));
       ClientProtocol nn = DFSUtil.createNamenode(
       ClientProtocol nn = DFSUtil.createNamenode(
-          NameNode.getAddress(token.getService().toString()), conf,
+          SecurityUtil.getTokenServiceAddr(delToken), conf,
           UserGroupInformation.getCurrentUser());
           UserGroupInformation.getCurrentUser());
       try {
       try {
         nn.cancelDelegationToken(delToken);
         nn.cancelDelegationToken(delToken);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -1538,7 +1538,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
    * write pipeline have failed. 
    * write pipeline have failed. 
    * @return the number of valid replicas of the current block
    * @return the number of valid replicas of the current block
    */
    */
-  synchronized int getNumCurrentReplicas() throws IOException {
+  public synchronized int getNumCurrentReplicas() throws IOException {
     dfsClient.checkOpen();
     dfsClient.checkOpen();
     isClosed();
     isClosed();
     if (streamer == null) {
     if (streamer == null) {

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -780,19 +780,6 @@ public class DFSUtil {
     return capacity <= 0 ? 0 : (remaining * 100.0f)/capacity; 
     return capacity <= 0 ? 0 : (remaining * 100.0f)/capacity; 
   }
   }
 
 
-  /**
-   * @param address address of format host:port
-   * @return InetSocketAddress for the address
-   */
-  public static InetSocketAddress getSocketAddress(String address) {
-    int colon = address.indexOf(":");
-    if (colon < 0) {
-      return new InetSocketAddress(address, 0);
-    }
-    return new InetSocketAddress(address.substring(0, colon), 
-        Integer.parseInt(address.substring(colon + 1)));
-  }
-
   /**
   /**
    * Round bytes to GiB (gibibyte)
    * Round bytes to GiB (gibibyte)
    * @param bytes number of bytes
    * @param bytes number of bytes

+ 1 - 36
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -107,45 +107,10 @@ public class DistributedFileSystem extends FileSystem {
     }
     }
 
 
     this.dfs = new DFSClient(uri, conf, statistics);
     this.dfs = new DFSClient(uri, conf, statistics);
-    this.uri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" + uri.getAuthority());
+    this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
     this.workingDir = getHomeDirectory();
     this.workingDir = getHomeDirectory();
   }
   }
 
 
-  /** Permit paths which explicitly specify the default port. */
-  @Override
-  protected void checkPath(Path path) {
-    URI thisUri = this.getUri();
-    URI thatUri = path.toUri();
-    String thatAuthority = thatUri.getAuthority();
-    if (thatUri.getScheme() != null
-        && thatUri.getScheme().equalsIgnoreCase(thisUri.getScheme())
-        && thatUri.getPort() == NameNode.DEFAULT_PORT
-        && (thisUri.getPort() == -1 || 
-            thisUri.getPort() == NameNode.DEFAULT_PORT)
-        && thatAuthority.substring(0,thatAuthority.indexOf(":"))
-        .equalsIgnoreCase(thisUri.getAuthority()))
-      return;
-    super.checkPath(path);
-  }
-
-  /** Normalize paths that explicitly specify the default port. */
-  @Override
-  public Path makeQualified(Path path) {
-    URI thisUri = this.getUri();
-    URI thatUri = path.toUri();
-    String thatAuthority = thatUri.getAuthority();
-    if (thatUri.getScheme() != null
-        && thatUri.getScheme().equalsIgnoreCase(thisUri.getScheme())
-        && thatUri.getPort() == NameNode.DEFAULT_PORT
-        && thisUri.getPort() == -1
-        && thatAuthority.substring(0,thatAuthority.indexOf(":"))
-        .equalsIgnoreCase(thisUri.getAuthority())) {
-      path = new Path(thisUri.getScheme(), thisUri.getAuthority(),
-                      thatUri.getPath());
-    }
-    return super.makeQualified(path);
-  }
-
   @Override
   @Override
   public Path getWorkingDirectory() {
   public Path getWorkingDirectory() {
     return workingDir;
     return workingDir;

+ 76 - 104
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.ServletUtil;
 import org.xml.sax.Attributes;
 import org.xml.sax.Attributes;
@@ -89,17 +90,20 @@ public class HftpFileSystem extends FileSystem
 
 
   public static final Text TOKEN_KIND = new Text("HFTP delegation");
   public static final Text TOKEN_KIND = new Text("HFTP delegation");
 
 
-  private String nnHttpUrl;
-  private Text hdfsServiceName;
+  protected UserGroupInformation ugi;
   private URI hftpURI;
   private URI hftpURI;
+
   protected InetSocketAddress nnAddr;
   protected InetSocketAddress nnAddr;
-  protected UserGroupInformation ugi; 
+  protected InetSocketAddress nnSecureAddr;
 
 
   public static final String HFTP_TIMEZONE = "UTC";
   public static final String HFTP_TIMEZONE = "UTC";
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
+
   private Token<?> delegationToken;
   private Token<?> delegationToken;
   private Token<?> renewToken;
   private Token<?> renewToken;
-  
+  private static final HftpDelegationTokenSelector hftpTokenSelector =
+      new HftpDelegationTokenSelector();
+
   public static final SimpleDateFormat getDateFormat() {
   public static final SimpleDateFormat getDateFormat() {
     final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT);
     final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT);
     df.setTimeZone(TimeZone.getTimeZone(HFTP_TIMEZONE));
     df.setTimeZone(TimeZone.getTimeZone(HFTP_TIMEZONE));
@@ -115,11 +119,8 @@ public class HftpFileSystem extends FileSystem
 
 
   @Override
   @Override
   protected int getDefaultPort() {
   protected int getDefaultPort() {
-    return getDefaultSecurePort();
-
-    //TODO: un-comment the following once HDFS-7510 is committed. 
-//    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
-//        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
+    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
+        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
   }
   }
 
 
   protected int getDefaultSecurePort() {
   protected int getDefaultSecurePort() {
@@ -127,16 +128,22 @@ public class HftpFileSystem extends FileSystem
         DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT);
         DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT);
   }
   }
 
 
-  @Override
-  public String getCanonicalServiceName() {
-    return SecurityUtil.buildDTServiceName(hftpURI, getDefaultPort());
+  protected InetSocketAddress getNamenodeAddr(URI uri) {
+    // use authority so user supplied uri can override port
+    return NetUtils.createSocketAddr(uri.getAuthority(), getDefaultPort());
   }
   }
-  
-  private String buildUri(String schema, String host, int port) {
-    StringBuilder sb = new StringBuilder(schema);
-    return sb.append(host).append(":").append(port).toString();
+
+  protected InetSocketAddress getNamenodeSecureAddr(URI uri) {
+    // must only use the host and the configured https port
+    return NetUtils.createSocketAddrForHost(uri.getHost(), getDefaultSecurePort());
   }
   }
 
 
+  @Override
+  public String getCanonicalServiceName() {
+    // unlike other filesystems, hftp's service is the secure port, not the
+    // actual port in the uri
+    return SecurityUtil.buildTokenService(nnSecureAddr).toString();
+  }
 
 
   @Override
   @Override
   public void initialize(final URI name, final Configuration conf)
   public void initialize(final URI name, final Configuration conf)
@@ -144,95 +151,51 @@ public class HftpFileSystem extends FileSystem
     super.initialize(name, conf);
     super.initialize(name, conf);
     setConf(conf);
     setConf(conf);
     this.ugi = UserGroupInformation.getCurrentUser(); 
     this.ugi = UserGroupInformation.getCurrentUser(); 
-    nnAddr = NetUtils.createSocketAddr(name.toString());
-    
-    // in case we open connection to hftp of a different cluster
-    // we need to know this cluster https port
-    // if it is not set we assume it is the same cluster or same port
-    int urlPort = conf.getInt("dfs.hftp.https.port", -1);
-    if(urlPort == -1)
-      urlPort = conf.getInt(DFSConfigKeys.DFS_HTTPS_PORT_KEY, 
-          DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT);
-
-    String normalizedNN = NetUtils.normalizeHostName(name.getHost());
-    nnHttpUrl = buildUri("https://", normalizedNN ,urlPort);
-    LOG.debug("using url to get DT:" + nnHttpUrl);
+    this.nnAddr = getNamenodeAddr(name);
+    this.nnSecureAddr = getNamenodeSecureAddr(name);
     try {
     try {
-      hftpURI = new URI(buildUri("hftp://", normalizedNN, urlPort));
-    } catch (URISyntaxException ue) {
-      throw new IOException("bad uri for hdfs", ue);
+      this.hftpURI = new URI(name.getScheme(), name.getAuthority(),
+                             null, null, null);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
     }
     }
 
 
-    // if one uses RPC port different from the Default one,  
-    // one should specify what is the setvice name for this delegation token
-    // otherwise it is hostname:RPC_PORT
-    String key = DelegationTokenSelector.SERVICE_NAME_KEY
-        + SecurityUtil.buildDTServiceName(name,
-            DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Trying to find DT for " + name + " using key=" + key + 
-          "; conf=" + conf.get(key, ""));
-    }
-    String nnServiceName = conf.get(key);
-    int nnPort = NameNode.DEFAULT_PORT;
-    if (nnServiceName != null) { // get the real port
-      nnPort = NetUtils.createSocketAddr(nnServiceName, 
-          NameNode.DEFAULT_PORT).getPort();
+    if (UserGroupInformation.isSecurityEnabled()) {
+      initDelegationToken();
     }
     }
-    try {
-      URI hdfsURI = new URI("hdfs://" + normalizedNN + ":" + nnPort);
-      hdfsServiceName = new Text(SecurityUtil.buildDTServiceName(hdfsURI, 
-                                                                 nnPort));
-    } catch (URISyntaxException ue) {
-      throw new IOException("bad uri for hdfs", ue);
+  }
+
+  protected void initDelegationToken() throws IOException {
+    // look for hftp token, then try hdfs
+    Token<?> token = selectHftpDelegationToken();
+    if (token == null) {
+      token = selectHdfsDelegationToken();
+    }  
+
+    // if we don't already have a token, go get one over https
+    boolean createdToken = false;
+    if (token == null) {
+      token = getDelegationToken(null);
+      createdToken = (token != null);
     }
     }
 
 
-    if (UserGroupInformation.isSecurityEnabled()) {
-      //try finding a token for this namenode (esp applicable for tasks
-      //using hftp). If there exists one, just set the delegationField
-      String hftpServiceName = getCanonicalServiceName();
-      for (Token<? extends TokenIdentifier> t : ugi.getTokens()) {
-        Text kind = t.getKind();
-        if (DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind)) {
-          if (t.getService().equals(hdfsServiceName)) {
-            setDelegationToken(t);
-            break;
-          }
-        } else if (TOKEN_KIND.equals(kind)) {
-          if (hftpServiceName
-              .equals(normalizeService(t.getService().toString()))) {
-            setDelegationToken(t);
-            break;
-          }
-        }
-      }
-      
-      //since we don't already have a token, go get one over https
-      if (delegationToken == null) {
-        setDelegationToken(getDelegationToken(null));
+    // we already had a token or getDelegationToken() didn't fail.
+    if (token != null) {
+      setDelegationToken(token);
+      if (createdToken) {
         dtRenewer.addRenewAction(this);
         dtRenewer.addRenewAction(this);
+        LOG.debug("Created new DT for " + token.getService());
+      } else {
+        LOG.debug("Found existing DT for " + token.getService());
       }
       }
     }
     }
   }
   }
 
 
-  private String normalizeService(String service) {
-    int colonIndex = service.indexOf(':');
-    if (colonIndex == -1) {
-      throw new IllegalArgumentException("Invalid service for hftp token: " + 
-                                         service);
-    }
-    String hostname = 
-        NetUtils.normalizeHostName(service.substring(0, colonIndex));
-    String port = service.substring(colonIndex + 1);
-    return hostname + ":" + port;
+  protected Token<DelegationTokenIdentifier> selectHftpDelegationToken() {
+    Text serviceName = SecurityUtil.buildTokenService(nnSecureAddr);
+    return hftpTokenSelector.selectToken(serviceName, ugi.getTokens());
   }
   }
 
 
-  //TODO: un-comment the following once HDFS-7510 is committed. 
-//  protected Token<DelegationTokenIdentifier> selectHftpDelegationToken() {
-//    Text serviceName = SecurityUtil.buildTokenService(nnSecureAddr);
-//    return hftpTokenSelector.selectToken(serviceName, ugi.getTokens());      
-//  }
-  
   protected Token<DelegationTokenIdentifier> selectHdfsDelegationToken() {
   protected Token<DelegationTokenIdentifier> selectHdfsDelegationToken() {
     return  DelegationTokenSelector.selectHdfsDelegationToken(
     return  DelegationTokenSelector.selectHdfsDelegationToken(
         nnAddr, ugi, getConf());
         nnAddr, ugi, getConf());
@@ -245,13 +208,17 @@ public class HftpFileSystem extends FileSystem
   }
   }
 
 
   @Override
   @Override
-  public <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
+  public synchronized <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
     renewToken = token;
     renewToken = token;
     // emulate the 203 usage of the tokens
     // emulate the 203 usage of the tokens
     // by setting the kind and service as if they were hdfs tokens
     // by setting the kind and service as if they were hdfs tokens
     delegationToken = new Token<T>(token);
     delegationToken = new Token<T>(token);
+    // NOTE: the remote nn must be configured to use hdfs
     delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
     delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
-    delegationToken.setService(hdfsServiceName);
+    // no need to change service because we aren't exactly sure what it
+    // should be.  we can guess, but it might be wrong if the local conf
+    // value is incorrect.  the service is a client side field, so the remote
+    // end does not care about the value
   }
   }
 
 
   @Override
   @Override
@@ -262,6 +229,7 @@ public class HftpFileSystem extends FileSystem
       ugi.reloginFromKeytab();
       ugi.reloginFromKeytab();
       return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
       return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
         public Token<?> run() throws IOException {
         public Token<?> run() throws IOException {
+          final String nnHttpUrl = DFSUtil.createUri("https", nnSecureAddr).toString();
           Credentials c;
           Credentials c;
           try {
           try {
             c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
             c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
@@ -291,12 +259,7 @@ public class HftpFileSystem extends FileSystem
 
 
   @Override
   @Override
   public URI getUri() {
   public URI getUri() {
-    try {
-      return new URI("hftp", null, nnAddr.getHostName(), nnAddr.getPort(),
-                     null, null, null);
-    } catch (URISyntaxException e) {
-      return null;
-    } 
+    return hftpURI;
   }
   }
 
 
   /**
   /**
@@ -722,11 +685,12 @@ public class HftpFileSystem extends FileSystem
     public long renew(Token<?> token, 
     public long renew(Token<?> token, 
                       Configuration conf) throws IOException {
                       Configuration conf) throws IOException {
       // update the kerberos credentials, if they are coming from a keytab
       // update the kerberos credentials, if they are coming from a keytab
-      UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
+      UserGroupInformation.getLoginUser().reloginFromKeytab();
       // use https to renew the token
       // use https to renew the token
+      InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
       return 
       return 
         DelegationTokenFetcher.renewDelegationToken
         DelegationTokenFetcher.renewDelegationToken
-        ("https://" + token.getService().toString(), 
+        (DFSUtil.createUri("https", serviceAddr).toString(), 
          (Token<DelegationTokenIdentifier>) token);
          (Token<DelegationTokenIdentifier>) token);
     }
     }
 
 
@@ -737,10 +701,18 @@ public class HftpFileSystem extends FileSystem
       // update the kerberos credentials, if they are coming from a keytab
       // update the kerberos credentials, if they are coming from a keytab
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
       // use https to cancel the token
       // use https to cancel the token
+      InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
       DelegationTokenFetcher.cancelDelegationToken
       DelegationTokenFetcher.cancelDelegationToken
-        ("https://" + token.getService().toString(), 
+        (DFSUtil.createUri("https", serviceAddr).toString(), 
          (Token<DelegationTokenIdentifier>) token);
          (Token<DelegationTokenIdentifier>) token);
+    }    
+  }
+  
+  private static class HftpDelegationTokenSelector
+  extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
+
+    public HftpDelegationTokenSelector() {
+      super(TOKEN_KIND);
     }
     }
-    
   }
   }
 }
 }

+ 11 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URL;
@@ -120,6 +121,16 @@ public class HsftpFileSystem extends HftpFileSystem {
     }
     }
   }
   }
 
 
+  @Override
+  protected int getDefaultPort() {
+    return getDefaultSecurePort();
+  }
+
+  @Override
+  protected InetSocketAddress getNamenodeSecureAddr(URI uri) {
+    return getNamenodeAddr(uri);
+  }
+
   @Override
   @Override
   protected HttpURLConnection openConnection(String path, String query)
   protected HttpURLConnection openConnection(String path, String query)
       throws IOException {
       throws IOException {
@@ -161,16 +172,6 @@ public class HsftpFileSystem extends HftpFileSystem {
     return (HttpURLConnection) conn;
     return (HttpURLConnection) conn;
   }
   }
 
 
-  @Override
-  public URI getUri() {
-    try {
-      return new URI("hsftp", null, nnAddr.getHostName(), nnAddr.getPort(),
-          null, null, null);
-    } catch (URISyntaxException e) {
-      return null;
-    }
-  }
-
   /**
   /**
    * Dummy hostname verifier that is used to bypass hostname checking
    * Dummy hostname verifier that is used to bypass hostname checking
    */
    */

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -49,10 +49,11 @@ public interface ClientDatanodeProtocol extends VersionedProtocol {
    * DN server side to insulate from the protocol serialization.
    * DN server side to insulate from the protocol serialization.
    * 
    * 
    * If you are adding/changing DN's interface then you need to 
    * If you are adding/changing DN's interface then you need to 
-   * change both this class and ALSO
-   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientDatanodeWireProtocol}.
-   * These changes need to be done in a compatible fashion as described in 
-   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientDatanodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    * 
    * 
    * The log of historical changes can be retrieved from the svn).
    * The log of historical changes can be retrieved from the svn).
    * 9: Added deleteBlockPool method
    * 9: Added deleteBlockPool method

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -73,11 +73,12 @@ public interface ClientProtocol extends VersionedProtocol {
    * This class is used by both the DFSClient and the 
    * This class is used by both the DFSClient and the 
    * NN server side to insulate from the protocol serialization.
    * NN server side to insulate from the protocol serialization.
    * 
    * 
-   * If you are adding/changing NN's interface then you need to 
-   * change both this class and ALSO
-   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}.
-   * These changes need to be done in a compatible fashion as described in 
-   * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
+   * If you are adding/changing this interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientNamenodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    * 
    * 
    * The log of historical changes can be retrieved from the svn).
    * The log of historical changes can be retrieved from the svn).
    * 69: Eliminate overloaded method names.
    * 69: Eliminate overloaded method names.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -73,7 +73,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public RefreshNamenodesResponseProto refreshNamenode(
+  public RefreshNamenodesResponseProto refreshNamenodes(
       RpcController unused, RefreshNamenodesRequestProto request)
       RpcController unused, RefreshNamenodesRequestProto request)
       throws ServiceException {
       throws ServiceException {
     try {
     try {

+ 12 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -43,8 +43,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
@@ -60,7 +63,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class ClientDatanodeProtocolTranslatorPB implements
 public class ClientDatanodeProtocolTranslatorPB implements
-    ClientDatanodeProtocol, Closeable {
+    ProtocolMetaInterface, ClientDatanodeProtocol, Closeable {
   public static final Log LOG = LogFactory
   public static final Log LOG = LogFactory
       .getLog(ClientDatanodeProtocolTranslatorPB.class);
       .getLog(ClientDatanodeProtocolTranslatorPB.class);
   
   
@@ -168,7 +171,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
   @Override
   @Override
   public void refreshNamenodes() throws IOException {
   public void refreshNamenodes() throws IOException {
     try {
     try {
-      rpcProxy.refreshNamenode(NULL_CONTROLLER, REFRESH_NAMENODES);
+      rpcProxy.refreshNamenodes(NULL_CONTROLLER, REFRESH_NAMENODES);
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
@@ -201,4 +204,11 @@ public class ClientDatanodeProtocolTranslatorPB implements
     return new BlockLocalPathInfo(PBHelper.convert(resp.getBlock()),
     return new BlockLocalPathInfo(PBHelper.convert(resp.getBlock()),
         resp.getLocalPath(), resp.getLocalMetaPath());
         resp.getLocalPath(), resp.getLocalMetaPath());
   }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), methodName);
+  }
 }
 }

+ 11 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -66,9 +66,12 @@ import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -125,8 +128,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSaf
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
 
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
@@ -139,7 +140,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class ClientNamenodeProtocolTranslatorPB implements
 public class ClientNamenodeProtocolTranslatorPB implements
-    ClientProtocol, Closeable, ProtocolTranslator {
+    ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
   final private ClientNamenodeProtocolPB rpcProxy;
 
 
   private static ClientNamenodeProtocolPB createNamenode(
   private static ClientNamenodeProtocolPB createNamenode(
@@ -876,6 +877,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
     }
   }
   }
 
 
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientNamenodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
+  }
+
   @Override
   @Override
   public Object getUnderlyingProxyObject() {
   public Object getUnderlyingProxyObject() {
     return rpcProxy;
     return rpcProxy;

+ 59 - 30
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java

@@ -47,25 +47,33 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgra
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
-import org.apache.hadoop.hdfs.server.protocolR23Compatible.DatanodeWireProtocol;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
@@ -79,14 +87,14 @@ import com.google.protobuf.ServiceException;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 @InterfaceStability.Stable
-public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
-    Closeable {
+public class DatanodeProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, DatanodeProtocol, Closeable {
   
   
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
-  private final static RpcController NULL_CONTROLLER = null;
   private final DatanodeProtocolPB rpcProxy;
   private final DatanodeProtocolPB rpcProxy;
   private static final VersionRequestProto VERSION_REQUEST = 
   private static final VersionRequestProto VERSION_REQUEST = 
       VersionRequestProto.newBuilder().build();
       VersionRequestProto.newBuilder().build();
+  private final static RpcController NULL_CONTROLLER = null;
   
   
   public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
   public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
       Configuration conf) throws IOException {
       Configuration conf) throws IOException {
@@ -101,7 +109,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
       UserGroupInformation ugi) throws IOException {
       UserGroupInformation ugi) throws IOException {
     return RPC.getProxy(DatanodeProtocolPB.class,
     return RPC.getProxy(DatanodeProtocolPB.class,
         RPC.getProtocolVersion(DatanodeProtocolPB.class), nameNodeAddr, ugi,
         RPC.getProtocolVersion(DatanodeProtocolPB.class), nameNodeAddr, ugi,
-        conf, NetUtils.getSocketFactory(conf, DatanodeWireProtocol.class));
+        conf, NetUtils.getSocketFactory(conf, DatanodeProtocolPB.class));
   }
   }
 
 
   /** Create a {@link NameNode} proxy */
   /** Create a {@link NameNode} proxy */
@@ -150,13 +158,17 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
   }
   }
 
 
   @Override
   @Override
-  public DatanodeRegistration registerDatanode(DatanodeRegistration registration)
-      throws IOException {
-    RegisterDatanodeRequestProto req = RegisterDatanodeRequestProto
-        .newBuilder().setRegistration(PBHelper.convert(registration)).build();
+  public DatanodeRegistration registerDatanode(DatanodeRegistration registration,
+      DatanodeStorage[] storages) throws IOException {
+    RegisterDatanodeRequestProto.Builder builder = RegisterDatanodeRequestProto
+        .newBuilder().setRegistration(PBHelper.convert(registration));
+    for (DatanodeStorage s : storages) {
+      builder.addStorages(PBHelper.convert(s));
+    }
+    
     RegisterDatanodeResponseProto resp;
     RegisterDatanodeResponseProto resp;
     try {
     try {
-      resp = rpcProxy.registerDatanode(NULL_CONTROLLER, req);
+      resp = rpcProxy.registerDatanode(NULL_CONTROLLER, builder.build());
     } catch (ServiceException se) {
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
       throw ProtobufHelper.getRemoteException(se);
     }
     }
@@ -165,17 +177,19 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
 
 
   @Override
   @Override
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      int xmitsInProgress, int xceiverCount, int failedVolumes)
-      throws IOException {
-    HeartbeatRequestProto req = HeartbeatRequestProto.newBuilder()
-        .setRegistration(PBHelper.convert(registration)).setCapacity(capacity)
-        .setDfsUsed(dfsUsed).setRemaining(remaining)
-        .setBlockPoolUsed(blockPoolUsed).setXmitsInProgress(xmitsInProgress)
-        .setXceiverCount(xceiverCount).setFailedVolumes(failedVolumes).build();
+      StorageReport[] reports, int xmitsInProgress, int xceiverCount,
+      int failedVolumes) throws IOException {
+    HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration))
+        .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
+        .setFailedVolumes(failedVolumes);
+    for (StorageReport r : reports) {
+      builder.addReports(PBHelper.convert(r));
+    }
+    
     HeartbeatResponseProto resp;
     HeartbeatResponseProto resp;
     try {
     try {
-      resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, req);
+      resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());
     } catch (ServiceException se) {
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
       throw ProtobufHelper.getRemoteException(se);
     }
     }
@@ -190,19 +204,23 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
 
 
   @Override
   @Override
   public DatanodeCommand blockReport(DatanodeRegistration registration,
   public DatanodeCommand blockReport(DatanodeRegistration registration,
-      String poolId, long[] blocks) throws IOException {
+      String poolId, StorageBlockReport[] reports) throws IOException {
     BlockReportRequestProto.Builder builder = BlockReportRequestProto
     BlockReportRequestProto.Builder builder = BlockReportRequestProto
         .newBuilder().setRegistration(PBHelper.convert(registration))
         .newBuilder().setRegistration(PBHelper.convert(registration))
         .setBlockPoolId(poolId);
         .setBlockPoolId(poolId);
-    if (blocks != null) {
+    
+    for (StorageBlockReport r : reports) {
+      StorageBlockReportProto.Builder reportBuilder = StorageBlockReportProto
+          .newBuilder().setStorageID(r.getStorageID());
+      long[] blocks = r.getBlocks();
       for (int i = 0; i < blocks.length; i++) {
       for (int i = 0; i < blocks.length; i++) {
-        builder.addBlocks(blocks[i]);
+        reportBuilder.addBlocks(blocks[i]);
       }
       }
+      builder.addReports(reportBuilder.build());
     }
     }
-    BlockReportRequestProto req = builder.build();
     BlockReportResponseProto resp;
     BlockReportResponseProto resp;
     try {
     try {
-      resp = rpcProxy.blockReport(NULL_CONTROLLER, req);
+      resp = rpcProxy.blockReport(NULL_CONTROLLER, builder.build());
     } catch (ServiceException se) {
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
       throw ProtobufHelper.getRemoteException(se);
     }
     }
@@ -211,20 +229,23 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
 
 
   @Override
   @Override
   public void blockReceivedAndDeleted(DatanodeRegistration registration,
   public void blockReceivedAndDeleted(DatanodeRegistration registration,
-      String poolId, ReceivedDeletedBlockInfo[] receivedAndDeletedBlocks)
+      String poolId, StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks)
       throws IOException {
       throws IOException {
     BlockReceivedAndDeletedRequestProto.Builder builder = 
     BlockReceivedAndDeletedRequestProto.Builder builder = 
         BlockReceivedAndDeletedRequestProto.newBuilder()
         BlockReceivedAndDeletedRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
         .setRegistration(PBHelper.convert(registration))
         .setBlockPoolId(poolId);
         .setBlockPoolId(poolId);
-    if (receivedAndDeletedBlocks != null) {
-      for (int i = 0; i < receivedAndDeletedBlocks.length; i++) {
-        builder.addBlocks(PBHelper.convert(receivedAndDeletedBlocks[i]));
+    for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) {
+      StorageReceivedDeletedBlocksProto.Builder repBuilder = 
+          StorageReceivedDeletedBlocksProto.newBuilder();
+      repBuilder.setStorageID(storageBlock.getStorageID());
+      for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
+        repBuilder.addBlocks(PBHelper.convert(rdBlock));
       }
       }
+      builder.addBlocks(repBuilder.build());
     }
     }
-    BlockReceivedAndDeletedRequestProto req = builder.build();
     try {
     try {
-      rpcProxy.blockReceivedAndDeleted(NULL_CONTROLLER, req);
+      rpcProxy.blockReceivedAndDeleted(NULL_CONTROLLER, builder.build());
     } catch (ServiceException se) {
     } catch (ServiceException se) {
       throw ProtobufHelper.getRemoteException(se);
       throw ProtobufHelper.getRemoteException(se);
     }
     }
@@ -301,4 +322,12 @@ public class DatanodeProtocolClientSideTranslatorPB implements DatanodeProtocol,
       throw ProtobufHelper.getRemoteException(se);
       throw ProtobufHelper.getRemoteException(se);
     }
     }
   }
   }
+
+  @Override // ProtocolMetaInterface
+  public boolean isMethodSupported(String methodName)
+      throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, DatanodeProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(DatanodeProtocolPB.class), methodName);
+  }
 }
 }

+ 46 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportR
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
@@ -41,6 +40,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterData
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
@@ -49,9 +51,13 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
@@ -85,8 +91,12 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     DatanodeRegistration registration = PBHelper.convert(request
     DatanodeRegistration registration = PBHelper.convert(request
         .getRegistration());
         .getRegistration());
     DatanodeRegistration registrationResp;
     DatanodeRegistration registrationResp;
+    DatanodeStorage[] storages = new DatanodeStorage[request.getStoragesCount()];
+    for (int i = 0; i < request.getStoragesCount(); i++) {
+      storages[i] = PBHelper.convert(request.getStorages(i));
+    }
     try {
     try {
-      registrationResp = impl.registerDatanode(registration);
+      registrationResp = impl.registerDatanode(registration, storages);
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
@@ -99,10 +109,17 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       HeartbeatRequestProto request) throws ServiceException {
       HeartbeatRequestProto request) throws ServiceException {
     HeartbeatResponse response;
     HeartbeatResponse response;
     try {
     try {
+      List<StorageReportProto> list = request.getReportsList();
+      StorageReport[] report = new StorageReport[list.size()];
+      int i = 0;
+      for (StorageReportProto p : list) {
+        report[i++] = new StorageReport(p.getStorageID(), p.getFailed(),
+            p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+            p.getBlockPoolUsed());
+      }
       response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
       response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
-          request.getCapacity(), request.getDfsUsed(), request.getRemaining(),
-          request.getBlockPoolUsed(), request.getXmitsInProgress(),
-          request.getXceiverCount(), request.getFailedVolumes());
+          report, request.getXmitsInProgress(), request.getXceiverCount(),
+          request.getFailedVolumes());
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
@@ -124,14 +141,21 @@ public class DatanodeProtocolServerSideTranslatorPB implements
   public BlockReportResponseProto blockReport(RpcController controller,
   public BlockReportResponseProto blockReport(RpcController controller,
       BlockReportRequestProto request) throws ServiceException {
       BlockReportRequestProto request) throws ServiceException {
     DatanodeCommand cmd = null;
     DatanodeCommand cmd = null;
-    List<Long> blockIds = request.getBlocksList();
-    long[] blocks = new long[blockIds.size()];
-    for (int i = 0; i < blockIds.size(); i++) {
-      blocks[i] = blockIds.get(i);
+    StorageBlockReport[] report = 
+        new StorageBlockReport[request.getReportsCount()];
+    
+    int index = 0;
+    for (StorageBlockReportProto s : request.getReportsList()) {
+      List<Long> blockIds = s.getBlocksList();
+      long[] blocks = new long[blockIds.size()];
+      for (int i = 0; i < blockIds.size(); i++) {
+        blocks[i] = blockIds.get(i);
+      }
+      report[index++] = new StorageBlockReport(s.getStorageID(), blocks);
     }
     }
     try {
     try {
       cmd = impl.blockReport(PBHelper.convert(request.getRegistration()),
       cmd = impl.blockReport(PBHelper.convert(request.getRegistration()),
-          request.getBlockPoolId(), blocks);
+          request.getBlockPoolId(), report);
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
@@ -147,11 +171,18 @@ public class DatanodeProtocolServerSideTranslatorPB implements
   public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
   public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
       RpcController controller, BlockReceivedAndDeletedRequestProto request)
       RpcController controller, BlockReceivedAndDeletedRequestProto request)
       throws ServiceException {
       throws ServiceException {
-    List<ReceivedDeletedBlockInfoProto> rdbip = request.getBlocksList();
-    ReceivedDeletedBlockInfo[] info = 
-        new ReceivedDeletedBlockInfo[rdbip.size()];
-    for (int i = 0; i < rdbip.size(); i++) {
-      info[i] = PBHelper.convert(rdbip.get(i));
+    List<StorageReceivedDeletedBlocksProto> sBlocks = request.getBlocksList();
+    StorageReceivedDeletedBlocks[] info = 
+        new StorageReceivedDeletedBlocks[sBlocks.size()];
+    for (int i = 0; i < sBlocks.size(); i++) {
+      StorageReceivedDeletedBlocksProto sBlock = sBlocks.get(i);
+      List<ReceivedDeletedBlockInfoProto> list = sBlock.getBlocksList();
+      ReceivedDeletedBlockInfo[] rdBlocks = 
+          new ReceivedDeletedBlockInfo[list.size()];
+      for (int j = 0; j < list.size(); j++) {
+        rdBlocks[j] = PBHelper.convert(list.get(j));
+      }
+      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageID(), rdBlocks);
     }
     }
     try {
     try {
       impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),
       impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java

@@ -29,8 +29,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
@@ -39,7 +42,7 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
 public class GetUserMappingsProtocolClientSideTranslatorPB implements
 public class GetUserMappingsProtocolClientSideTranslatorPB implements
-    GetUserMappingsProtocol, Closeable {
+    ProtocolMetaInterface, GetUserMappingsProtocol, Closeable {
 
 
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final static RpcController NULL_CONTROLLER = null;
@@ -86,4 +89,11 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements
     }
     }
     return resp.getGroupsList().toArray(new String[resp.getGroupsCount()]);
     return resp.getGroupsList().toArray(new String[resp.getGroupsCount()]);
   }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        GetUserMappingsProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(GetUserMappingsProtocolPB.class), methodName);
+  }
 }
 }

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java

@@ -37,8 +37,11 @@ import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.RpcController;
@@ -52,7 +55,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class InterDatanodeProtocolTranslatorPB implements
 public class InterDatanodeProtocolTranslatorPB implements
-    InterDatanodeProtocol, Closeable {
+    ProtocolMetaInterface, InterDatanodeProtocol, Closeable {
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final static RpcController NULL_CONTROLLER = null;
   final private InterDatanodeProtocolPB rpcProxy;
   final private InterDatanodeProtocolPB rpcProxy;
@@ -116,4 +119,11 @@ public class InterDatanodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        InterDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(InterDatanodeProtocolPB.class), methodName);
+  }
 }
 }

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java

@@ -31,8 +31,11 @@ import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
@@ -44,7 +47,8 @@ import com.google.protobuf.ServiceException;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 @InterfaceStability.Stable
-public class JournalProtocolTranslatorPB implements JournalProtocol, Closeable {
+public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
+    JournalProtocol, Closeable {
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final static RpcController NULL_CONTROLLER = null;
   private final JournalProtocolPB rpcProxy;
   private final JournalProtocolPB rpcProxy;
@@ -103,4 +107,11 @@ public class JournalProtocolTranslatorPB implements JournalProtocol, Closeable {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, JournalProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(JournalProtocolPB.class), methodName);
+  }
 }
 }

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java

@@ -20,17 +20,12 @@ package org.apache.hadoop.hdfs.protocolPB;
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
@@ -45,21 +40,19 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpo
 import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
@@ -74,7 +67,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
 public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
-    Closeable {
+    ProtocolMetaInterface, Closeable {
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final static RpcController NULL_CONTROLLER = null;
   
   
@@ -241,4 +234,11 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, NamenodeProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(NamenodeProtocolPB.class), methodName);
+  }
 }
 }

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@@ -52,11 +53,14 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommand
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto.StorageState;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -109,6 +113,8 @@ import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
@@ -121,6 +127,7 @@ import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -1301,4 +1308,41 @@ public class PBHelper {
     builder.setTxid(hb.getTxId());
     builder.setTxid(hb.getTxId());
     return builder.build();
     return builder.build();
   }
   }
+
+  public static DatanodeStorageProto convert(DatanodeStorage s) {
+    return DatanodeStorageProto.newBuilder()
+        .setState(PBHelper.convert(s.getState()))
+        .setStorageID(s.getStorageID()).build();
+  }
+
+  private static StorageState convert(State state) {
+    switch(state) {
+    case READ_ONLY:
+      return StorageState.READ_ONLY;
+    case NORMAL:
+    default:
+      return StorageState.NORMAL;
+    }
+  }
+
+  public static DatanodeStorage convert(DatanodeStorageProto s) {
+    return new DatanodeStorage(s.getStorageID(), PBHelper.convert(s.getState()));
+  }
+
+  private static State convert(StorageState state) {
+    switch(state) {
+    case READ_ONLY:
+      return DatanodeStorage.State.READ_ONLY;
+    case NORMAL:
+    default:
+      return DatanodeStorage.State.NORMAL;
+    }
+  }
+
+  public static StorageReportProto convert(StorageReport r) {
+    return StorageReportProto.newBuilder()
+        .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
+        .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
+        .setStorageID(r.getStorageID()).build();
+  }
 }
 }

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java

@@ -28,8 +28,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
@@ -38,7 +41,7 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
 public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
 public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
-    RefreshAuthorizationPolicyProtocol, Closeable {
+    ProtocolMetaInterface, RefreshAuthorizationPolicyProtocol, Closeable {
 
 
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final static RpcController NULL_CONTROLLER = null;
@@ -83,4 +86,13 @@ public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
       throw ProtobufHelper.getRemoteException(se);
       throw ProtobufHelper.getRemoteException(se);
     }
     }
   }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        RefreshAuthorizationPolicyProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
+        methodName);
+  }
 }
 }

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java

@@ -29,8 +29,11 @@ import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -39,7 +42,7 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
 public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
 public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
-    RefreshUserMappingsProtocol, Closeable {
+    ProtocolMetaInterface, RefreshUserMappingsProtocol, Closeable {
 
 
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final static RpcController NULL_CONTROLLER = null;
@@ -95,4 +98,13 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
       throw ProtobufHelper.getRemoteException(se);
       throw ProtobufHelper.getRemoteException(se);
     }
     }
   }
   }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil
+        .isMethodSupported(rpcProxy, RefreshUserMappingsProtocolPB.class,
+            RpcKind.RPC_PROTOCOL_BUFFER,
+            RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
+            methodName);
+  }
 }
 }

+ 0 - 111
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java

@@ -1,111 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**************************************************
- * A Block is a Hadoop FS primitive, identified by a long.
- **************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BlockWritable implements Writable {
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (BlockWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new BlockWritable(); }
-       });
-  }
-
-
-  private long blockId;
-  private long numBytes;
-  private long generationStamp;
-
-  public BlockWritable() {this(0, 0, 0);}
-
-  public BlockWritable(final long blkid, final long len, final long genStamp) {
-    this.blockId = blkid;
-    this.numBytes = len;
-    this.generationStamp = genStamp;
-  }
-
-  /////////////////////////////////////
-  // Writable
-  /////////////////////////////////////
-  @Override // Writable
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(blockId);
-    out.writeLong(numBytes);
-    out.writeLong(generationStamp);
-  }
-
-  @Override // Writable
-  public void readFields(DataInput in) throws IOException {
-    this.blockId = in.readLong();
-    this.numBytes = in.readLong();
-    this.generationStamp = in.readLong();
-  }
-
-  public static BlockWritable convert(Block b) {
-    return new BlockWritable(b.getBlockId(), b.getNumBytes(),
-        b.getGenerationStamp());
-  }
-
-  public Block convert() {
-    return new Block(blockId, numBytes, generationStamp);
-  }
-  
-  public long getBlockId() {
-    return blockId;
-  }
-
-  public long getNumBytes() {
-    return numBytes;
-  }
-
-  public long getGenerationStamp() {
-    return generationStamp;
-  }
-  
-  public static Block[] convert(BlockWritable[] blocks) {
-    Block[] ret = new Block[blocks.length];
-    for (int i = 0; i < blocks.length; i++) {
-      ret[i] = blocks[i].convert();
-    }
-    return ret;
-  }
-  
-  public static BlockWritable[] convert(Block[] blocks) {
-    BlockWritable[] ret = new BlockWritable[blocks.length];
-    for (int i = 0; i < blocks.length; i++) {
-      ret[i] = BlockWritable.convert(blocks[i]);
-    }
-    return ret;
-  }
-}

+ 0 - 129
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java

@@ -1,129 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
-import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-/** A class to implement an array of BlockLocations
- *  It provide efficient customized serialization/deserialization methods
- *  in stead of using the default array (de)serialization provided by RPC
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BlocksWithLocationsWritable implements Writable {
-
-  /**
-   * A class to keep track of a block and its locations
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  public static class BlockWithLocationsWritable  implements Writable {
-    private BlockWritable block;
-    private String datanodeIDs[];
-    
-    /** default constructor */
-    public BlockWithLocationsWritable() {
-      block = new BlockWritable();
-      datanodeIDs = null;
-    }
-    
-    /** constructor */
-    public BlockWithLocationsWritable(BlockWritable b, String[] datanodes) {
-      block = b;
-      datanodeIDs = datanodes;
-    }
-    
-    /** deserialization method */
-    public void readFields(DataInput in) throws IOException {
-      block.readFields(in);
-      int len = WritableUtils.readVInt(in); // variable length integer
-      datanodeIDs = new String[len];
-      for(int i=0; i<len; i++) {
-        datanodeIDs[i] = Text.readString(in);
-      }
-    }
-    
-    /** serialization method */
-    public void write(DataOutput out) throws IOException {
-      block.write(out);
-      WritableUtils.writeVInt(out, datanodeIDs.length); // variable length int
-      for(String id:datanodeIDs) {
-        Text.writeString(out, id);
-      }
-    }
-  }
-
-  private BlockWithLocationsWritable[] blocks;
-
-  /** default constructor */
-  BlocksWithLocationsWritable() {
-  }
-
-  /** Constructor with one parameter */
-  public BlocksWithLocationsWritable( BlockWithLocationsWritable[] blocks ) {
-    this.blocks = blocks;
-  }
-
-  /** serialization method */
-  public void write( DataOutput out ) throws IOException {
-    WritableUtils.writeVInt(out, blocks.length);
-    for(int i=0; i<blocks.length; i++) {
-      blocks[i].write(out);
-    }
-  }
-
-  /** deserialization method */
-  public void readFields(DataInput in) throws IOException {
-    int len = WritableUtils.readVInt(in);
-    blocks = new BlockWithLocationsWritable[len];
-    for(int i=0; i<len; i++) {
-      blocks[i] = new BlockWithLocationsWritable();
-      blocks[i].readFields(in);
-    }
-  }
-
-  public static BlocksWithLocationsWritable convert(BlocksWithLocations locs) {
-    BlockWithLocations[] blocks = locs.getBlocks();
-    BlockWithLocationsWritable[] blocksWritable = 
-        new BlockWithLocationsWritable[blocks.length];
-    for (int i = 0; i < blocks.length; i++) {
-      blocksWritable[i] = new BlockWithLocationsWritable(
-          BlockWritable.convert(blocks[i].getBlock()), blocks[i].getDatanodes());
-    }
-    return new BlocksWithLocationsWritable(blocksWritable);
-  }
-  
-  public BlocksWithLocations convert() {
-    BlockWithLocations[] locs = new BlockWithLocations[blocks.length];
-    for (int i = 0; i < blocks.length; i++) {
-      locs[i] = new BlockWithLocations(blocks[i].block.convert(),
-          blocks[i].datanodeIDs);
-    }
-    return new BlocksWithLocations(locs);
-  }
-}

+ 0 - 92
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CheckpointSignatureWritable.java

@@ -1,92 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * A unique signature intended to identify checkpoint transactions.
- */
-@InterfaceAudience.Private
-public class CheckpointSignatureWritable implements Writable { 
-  private String blockpoolID = "";
-  private long mostRecentCheckpointTxId;
-  private long curSegmentTxId;
-  private StorageInfoWritable storageInfo;
-
-  public CheckpointSignatureWritable() {}
-
-  CheckpointSignatureWritable(long mostRecentCheckpointTxId,
-      long curSegmentTxId, int layoutVersion, int namespaceID, String bpid,
-      String clusterID, long cTime) {
-    this.blockpoolID = bpid;
-    this.mostRecentCheckpointTxId = mostRecentCheckpointTxId;
-    this.curSegmentTxId = curSegmentTxId;
-    this.storageInfo = new StorageInfoWritable(layoutVersion, namespaceID,
-        clusterID, cTime);
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {
-    WritableFactories.setFactory(CheckpointSignatureWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new CheckpointSignatureWritable();
-          }
-        });
-  }
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    storageInfo.write(out);
-    WritableUtils.writeString(out, blockpoolID);
-    out.writeLong(mostRecentCheckpointTxId);
-    out.writeLong(curSegmentTxId);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    storageInfo.readFields(in);
-    blockpoolID = WritableUtils.readString(in);
-    mostRecentCheckpointTxId = in.readLong();
-    curSegmentTxId = in.readLong();
-  }
-
-  public static CheckpointSignatureWritable convert(
-      CheckpointSignature sig) {
-    return new CheckpointSignatureWritable(sig.getMostRecentCheckpointTxId(),
-        sig.getCurSegmentTxId(), sig.getLayoutVersion(), sig.getNamespaceID(),
-        sig.getBlockpoolID(), sig.getClusterID(), sig.getCTime());
-  }
-
-  public CheckpointSignature convert() {
-    return new CheckpointSignature(storageInfo.convert(), blockpoolID,
-        mostRecentCheckpointTxId, curSegmentTxId);
-  }
-}

+ 0 - 129
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolServerSideTranslatorR23.java

@@ -1,129 +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.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.token.Token;
-
-/**
- * This class is used on the server side.
- * Calls come across the wire for the protocol family of Release 23 onwards.
- * This class translates the R23 data types to the internal data types used
- * inside the DN as specified in the generic ClientDatanodeProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientDatanodeProtocolServerSideTranslatorR23 implements
-  ClientDatanodeWireProtocol {
-  
-  final private ClientDatanodeProtocol server;
-
-  /**
-   * 
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public ClientDatanodeProtocolServerSideTranslatorR23(
-      ClientDatanodeProtocol server) throws IOException {
-    this.server = server;
-  }
-  
-  /**
-   * the client side will redirect getProtocolSignature to 
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call
-   * getProtocolVersion and possibly in the future getProtocolSignature.
-   * Hence we still implement it even though the end client's call will
-   * never reach here.
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and
-     * signature is that of  {@link ClientDatanodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(
-        ClientDatanodeWireProtocol.class))) {
-      throw new IOException("Datanode Serverside implements " + 
-          ClientDatanodeWireProtocol.class + 
-          ". The following requested protocol is unknown: " + protocol);
-    }
-    
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-        ClientDatanodeWireProtocol.versionID, 
-        ClientDatanodeWireProtocol.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable 
-    getProtocolSignature2(
-        String protocol, long clientVersion, int clientMethodsHash)
-      throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and
-     * signature is that of  {@link ClientNamenodeProtocol}
-     */
-   return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
-    if (protocol.equals(RPC.getProtocolName(
-        ClientDatanodeWireProtocol.class))) {
-      return ClientDatanodeWireProtocol.versionID; 
-    }
-    throw new IOException("Datanode Serverside implements " + 
-        ClientDatanodeWireProtocol.class + 
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public long getReplicaVisibleLength(ExtendedBlockWritable b) throws IOException {
-    return 
-        server.getReplicaVisibleLength(ExtendedBlockWritable.convertExtendedBlock(b));
-  }
-
-  @Override
-  public void refreshNamenodes() throws IOException {
-    server.refreshNamenodes();
-  }
-
-  @Override
-  public void deleteBlockPool(String bpid, boolean force) throws IOException {
-    server.deleteBlockPool(bpid, force);
-  }
-
-  @Override
-  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
-      Token<BlockTokenIdentifier> token) throws IOException {
-    return server.getBlockLocalPathInfo(block, token);
-  }
-}

+ 0 - 162
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeProtocolTranslatorR23.java

@@ -1,162 +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.protocolR23Compatible;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import javax.net.SocketFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-
-
-/**
- * This class forwards ClientDatanodeProtocol calls as RPC to the DN server
- * while translating from the parameter types used in ClientDatanodeProtocol to
- * those used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientDatanodeProtocolTranslatorR23 implements 
-  ClientDatanodeProtocol {
-  
-  final private ClientDatanodeWireProtocol rpcProxy;
-  
-  public ClientDatanodeProtocolTranslatorR23(DatanodeID datanodeid,
-      Configuration conf, int socketTimeout, LocatedBlock locatedBlock)
-      throws IOException {
-    rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf, 
-                  socketTimeout, locatedBlock);
-  }
-  
-  /** used for testing */
-  public ClientDatanodeProtocolTranslatorR23(InetSocketAddress addr,
-      UserGroupInformation ticket,
-      Configuration conf,
-      SocketFactory factory) throws IOException {
-    rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory);
-  }
-  
-  /**
-   * Constructor.
-   * @param datanodeid Datanode to connect to.
-   * @param conf Configuration.
-   * @param socketTimeout Socket timeout to use.
-   * @throws IOException
-   */
-  public ClientDatanodeProtocolTranslatorR23(DatanodeID datanodeid,
-      Configuration conf, int socketTimeout) throws IOException {
-    InetSocketAddress addr = NetUtils.createSocketAddr(datanodeid.getHost()
-        + ":" + datanodeid.getIpcPort());
-    rpcProxy = RPC.getProxy(ClientDatanodeWireProtocol.class,
-        ClientDatanodeWireProtocol.versionID, addr,
-        UserGroupInformation.getCurrentUser(), conf,
-        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
-  }
-
-  static ClientDatanodeWireProtocol createClientDatanodeProtocolProxy(
-      DatanodeID datanodeid, Configuration conf, int socketTimeout,
-      LocatedBlock locatedBlock)
-      throws IOException {
-    InetSocketAddress addr = NetUtils.createSocketAddr(
-      datanodeid.getHost() + ":" + datanodeid.getIpcPort());
-    if (ClientDatanodeWireProtocol.LOG.isDebugEnabled()) {
-      ClientDatanodeWireProtocol.LOG.debug(
-          "ClientDatanodeProtocol addr=" + addr);
-    }
-    
-    // Since we're creating a new UserGroupInformation here, we know that no
-    // future RPC proxies will be able to re-use the same connection. And
-    // usages of this proxy tend to be one-off calls.
-    //
-    // This is a temporary fix: callers should really achieve this by using
-    // RPC.stopProxy() on the resulting object, but this is currently not
-    // working in trunk. See the discussion on HDFS-1965.
-    Configuration confWithNoIpcIdle = new Configuration(conf);
-    confWithNoIpcIdle.setInt(CommonConfigurationKeysPublic
-        .IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
-
-    UserGroupInformation ticket = UserGroupInformation
-        .createRemoteUser(locatedBlock.getBlock().getLocalBlock().toString());
-    ticket.addToken(locatedBlock.getBlockToken());
-    return RPC.getProxy(ClientDatanodeWireProtocol.class,
-      ClientDatanodeWireProtocol.versionID, addr, ticket, confWithNoIpcIdle,
-        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
-  }
-  
-  static ClientDatanodeWireProtocol createClientDatanodeProtocolProxy(
-      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
-      SocketFactory factory) throws IOException {
-    return RPC.getProxy(ClientDatanodeWireProtocol.class,
-        ClientDatanodeWireProtocol.versionID, addr, ticket, conf,
-        factory);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(
-      String protocolName, long clientVersion, int clientMethodHash)
-      throws IOException {
-    return ProtocolSignatureWritable.convert(
-        rpcProxy.getProtocolSignature2(
-            protocolName, clientVersion, clientMethodHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
-    return rpcProxy.getReplicaVisibleLength(
-        ExtendedBlockWritable.convertExtendedBlock(b));
-  }
-
-  @Override
-  public void refreshNamenodes() throws IOException {
-    rpcProxy.refreshNamenodes();
-
-  }
-
-  @Override
-  public void deleteBlockPool(String bpid, boolean force) throws IOException {
-    rpcProxy.deleteBlockPool(bpid, force);
-
-  }
-
-  @Override
-  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
-      Token<BlockTokenIdentifier> token) throws IOException {
-    return rpcProxy.getBlockLocalPathInfo(block, token);
-  }
-}

+ 0 - 100
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientDatanodeWireProtocol.java

@@ -1,100 +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.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
-
-/** 
- * This class defines the actual protocol used to communicate with the
- * DN via RPC using writable types.
- * The parameters in the methods which are specified in the
- * package are separate from those used internally in the DN and DFSClient
- * and hence need to be converted using {@link ClientDatanodeProtocolTranslatorR23}
- * and {@link ClientDatanodeProtocolServerSideTranslatorR23}.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
-@TokenInfo(BlockTokenSelector.class)
-@ProtocolInfo(protocolName = HdfsConstants.CLIENT_DATANODE_PROTOCOL_NAME)
-public interface ClientDatanodeWireProtocol extends VersionedProtocol {
-  public static final Log LOG = 
-      LogFactory.getLog(ClientDatanodeWireProtocol.class);
-
-  /**
-   * The  rules for changing this protocol are the same as that for
-   * {@link ClientNamenodeWireProtocol} - see that java file for details.
-   * 9: Added deleteBlockPool method
-   * 10 Moved the R23 protocol
-   */
-  public static final long versionID = 10L;
-
-  /**
-   * The specification of this method matches that of
-   * 
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol
-   * #getReplicaVisibleLength(org.apache.hadoop.hdfs.protocol.ExtendedBlock)}
-   */
-  long getReplicaVisibleLength(ExtendedBlockWritable b) throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#refreshNamenodes()}
-   */
-  void refreshNamenodes() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#deleteBlockPool(String, boolean)}
-   */
-  void deleteBlockPool(String bpid, boolean force) throws IOException; 
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
-   */
-  BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
-      Token<BlockTokenIdentifier> token) throws IOException;
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 to the method name
-   * to avoid conflict.
-   */
-  public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
-           getProtocolSignature2(String protocol, 
-      long clientVersion,
-      int clientMethodsHash) throws IOException;
-}

+ 0 - 463
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolServerSideTranslatorR23.java

@@ -1,463 +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.protocolR23Compatible;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
-
-/**
- * This class is used on the server side. Calls come across the wire for the
- * protocol family of Release 23 onwards. This class translates the R23 data
- * types to the native data types used inside the NN as specified in the generic
- * ClientProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientNamenodeProtocolServerSideTranslatorR23 implements
-    ClientNamenodeWireProtocol {
-  final private ClientProtocol server;
-
-  /**
-   * Constructor
-   * 
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public ClientNamenodeProtocolServerSideTranslatorR23(ClientProtocol server)
-      throws IOException {
-    this.server = server;
-  }
-
-  /**
-   * The client side will redirect getProtocolSignature to
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call getProtocolVersion
-   * and possibly in the future getProtocolSignature. Hence we still implement
-   * it even though the end client's call will never reach here.
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link ClientNamenodeProtocol}
-     * 
-     */
-    if (!protocol.equals(RPC.getProtocolName(
-        ClientNamenodeWireProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(ClientNamenodeWireProtocol.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        ClientNamenodeWireProtocol.versionID,
-        ClientNamenodeWireProtocol.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable 
-          getProtocolSignature2(
-      String protocol, long clientVersion, int clientMethodsHash)
-      throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link ClientNamenodeProtocol}
-     * 
-     */
-
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    if (protocol.equals(RPC.getProtocolName(
-        ClientNamenodeWireProtocol.class))) {
-      return ClientNamenodeWireProtocol.versionID;
-    }
-    throw new IOException("Namenode Serverside implements " +
-        RPC.getProtocolName(ClientNamenodeWireProtocol.class) +
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public LocatedBlocksWritable getBlockLocations(
-      String src, long offset, long length)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return LocatedBlocksWritable.convertLocatedBlocks(
-        server.getBlockLocations(src, offset, length));
-  }
-
-  @Override
-  public FsServerDefaultsWritable getServerDefaults() throws IOException {
-    return FsServerDefaultsWritable.convert(server.getServerDefaults());
-  }
-
-  @Override
-  public void create(String src, FsPermissionWritable masked, String clientName,
-      EnumSetWritable<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize) throws AccessControlException,
-      AlreadyBeingCreatedException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.create(src, FsPermissionWritable.convertPermission(masked),
-        clientName, flag, createParent, replication, blockSize);
-
-  }
-
-  @Override
-  public LocatedBlockWritable append(String src, String clientName)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.append(src, clientName));
-  }
-
-  @Override
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return server.setReplication(src, replication);
-  }
-
-  @Override
-  public void setPermission(String src, FsPermissionWritable permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.setPermission(src, 
-        FsPermissionWritable.convertPermission(permission));
-
-  }
-
-  @Override
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.setOwner(src, username, groupname);
-
-  }
-
-  @Override
-  public void abandonBlock(ExtendedBlockWritable b, String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    server.abandonBlock(
-        ExtendedBlockWritable.convertExtendedBlock(b), src, holder);
-
-  }
-
-  @Override
-  public LocatedBlockWritable addBlock(String src, String clientName,
-      ExtendedBlockWritable previous, DatanodeInfoWritable[] excludeNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.addBlock(src, clientName,
-        ExtendedBlockWritable.convertExtendedBlock(previous),
-        DatanodeInfoWritable.convertDatanodeInfo(excludeNodes)));
-  }
-
-  @Override
-  public LocatedBlockWritable getAdditionalDatanode(String src, ExtendedBlockWritable blk,
-      DatanodeInfoWritable[] existings, DatanodeInfoWritable[] excludes,
-      int numAdditionalNodes, String clientName) throws AccessControlException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.getAdditionalDatanode(src,
-              ExtendedBlockWritable.convertExtendedBlock(blk),
-              DatanodeInfoWritable.convertDatanodeInfo(existings),
-              DatanodeInfoWritable.convertDatanodeInfo(excludes),
-              numAdditionalNodes, clientName));
-  }
-
-  @Override
-  public boolean complete(String src, String clientName, ExtendedBlockWritable last)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return server.complete(src, clientName,
-        ExtendedBlockWritable.convertExtendedBlock(last));
-  }
-
-  @Override
-  public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException {
-    server.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
-
-  }
-
-  @Override
-  public boolean rename(String src, String dst) throws UnresolvedLinkException,
-      IOException {
-    return server.rename(src, dst);
-  }
-
-  @Override
-  public void concat(String trg, String[] srcs) throws IOException,
-      UnresolvedLinkException {
-    server.concat(trg, srcs);
-
-  }
-
-  @Override
-  public void rename2(String src, String dst, Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    server.rename2(src, dst, options);
-  }
-
-  @Override
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return server.delete(src, recursive);
-  }
-
-  @Override
-  public boolean mkdirs(String src, FsPermissionWritable masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-
-    return server.mkdirs(src, FsPermissionWritable.convertPermission(masked),
-        createParent);
-  }
-
-  @Override
-  public DirectoryListingWritable getListing(String src, byte[] startAfter,
-      boolean needLocation) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return DirectoryListingWritable.convertDirectoryListing(
-        server.getListing(src, startAfter, needLocation));
-  }
-
-  @Override
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException {
-    server.renewLease(clientName);
-
-  }
-
-  @Override
-  public boolean recoverLease(String src, String clientName) throws IOException {
-    return server.recoverLease(src, clientName);
-  }
-
-  @Override
-  public long[] getStats() throws IOException {
-    return server.getStats();
-  }
-
-  @Override
-  public DatanodeInfoWritable[] getDatanodeReport(DatanodeReportType type)
-      throws IOException {
-    return DatanodeInfoWritable
-        .convertDatanodeInfo(server.getDatanodeReport(type));
-  }
-
-  @Override
-  public long getPreferredBlockSize(String filename) throws IOException,
-      UnresolvedLinkException {
-    return server.getPreferredBlockSize(filename);
-  }
-
-  @Override
-  public boolean setSafeMode(SafeModeAction action) throws IOException {
-    return server.setSafeMode(action);
-  }
-
-  @Override
-  public void saveNamespace() throws AccessControlException, IOException {
-    server.saveNamespace();
-
-  }
-
-  @Override
-  public boolean restoreFailedStorage(String arg)
-      throws AccessControlException, IOException {
-    return server.restoreFailedStorage(arg);
-  }
-
-  @Override
-  public void refreshNodes() throws IOException {
-    server.refreshNodes();
-
-  }
-
-  @Override
-  public void finalizeUpgrade() throws IOException {
-    server.finalizeUpgrade();
-
-  }
-
-  @Override
-  public UpgradeStatusReportWritable distributedUpgradeProgress(UpgradeAction action)
-      throws IOException {
-    return UpgradeStatusReportWritable.convert(
-        server.distributedUpgradeProgress(action));
-  }
-
-  @Override
-  public CorruptFileBlocksWritable listCorruptFileBlocks(String path, String cookie)
-      throws IOException {
-    return CorruptFileBlocksWritable.convertCorruptFilesBlocks(
-        server.listCorruptFileBlocks(path, cookie));
-  }
-
-  @Override
-  public void metaSave(String filename) throws IOException {
-    server.metaSave(filename);
-
-  }
-
-  @Override
-  public HdfsFileStatusWritable getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable.convertHdfsFileStatus(
-        server.getFileInfo(src));
-  }
-
-  @Override
-  public HdfsFileStatusWritable getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable.convertHdfsFileStatus(
-        server.getFileLinkInfo(src));
-  }
-
-  @Override
-  public ContentSummaryWritable getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return ContentSummaryWritable.convert(server.getContentSummary(path));
-  }
-
-  @Override
-  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    server.setQuota(path, namespaceQuota, diskspaceQuota);
-
-  }
-
-  @Override
-  public void fsync(String src, String client) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    server.fsync(src, client);
-
-  }
-
-  @Override
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    server.setTimes(src, mtime, atime);
-
-  }
-
-  @Override
-  public void createSymlink(String target, String link, FsPermissionWritable dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    server.createSymlink(target, link, FsPermissionWritable.convertPermission(dirPerm),
-        createParent);
-
-  }
-
-  @Override
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException {
-    return server.getLinkTarget(path);
-  }
-
-  @Override
-  public LocatedBlockWritable updateBlockForPipeline(ExtendedBlockWritable block,
-      String clientName) throws IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        server.updateBlockForPipeline(
-        ExtendedBlockWritable.convertExtendedBlock(block), clientName));
-  }
-
-  @Override
-  public void updatePipeline(String clientName, ExtendedBlockWritable oldBlock,
-      ExtendedBlockWritable newBlock, DatanodeIDWritable[] newNodes)
-    throws IOException {
-    server.updatePipeline(clientName, 
-              ExtendedBlockWritable.convertExtendedBlock(oldBlock), 
-              ExtendedBlockWritable.convertExtendedBlock(newBlock),
-              DatanodeIDWritable.convertDatanodeID(newNodes));
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
-      throws IOException {
-    return server.getDelegationToken(renewer);
-  }
-
-  @Override
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    return server.renewDelegationToken(token);
-  }
-
-  @Override
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    server.cancelDelegationToken(token);
-  }
-
-  @Override
-  public void setBalancerBandwidth(long bandwidth) throws IOException {
-    server.setBalancerBandwidth(bandwidth);
-  }
-}

+ 0 - 480
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeProtocolTranslatorR23.java

@@ -1,480 +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.protocolR23Compatible;
-
-import java.io.Closeable;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.io.retry.RetryProxy;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
-/**
- * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
- * while translating from the parameter types used in ClientProtocol to those
- * used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientNamenodeProtocolTranslatorR23 implements
-    ClientProtocol, Closeable {
-  final private ClientNamenodeWireProtocol rpcProxy;
-
-  private static ClientNamenodeWireProtocol createNamenode(
-      InetSocketAddress nameNodeAddr, Configuration conf,
-      UserGroupInformation ugi) throws IOException {
-    return RPC.getProxy(ClientNamenodeWireProtocol.class,
-        ClientNamenodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
-        NetUtils.getSocketFactory(conf, ClientNamenodeWireProtocol.class));
-  }
-
-  /** Create a {@link NameNode} proxy */
-  static ClientNamenodeWireProtocol createNamenodeWithRetry(
-      ClientNamenodeWireProtocol rpcNamenode) {
-    RetryPolicy createPolicy = RetryPolicies
-        .retryUpToMaximumCountWithFixedSleep(5,
-            HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
-
-    Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap = new HashMap<Class<? extends Exception>, RetryPolicy>();
-    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
-        createPolicy);
-
-    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = 
-        new HashMap<Class<? extends Exception>, RetryPolicy>();
-    exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
-        .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
-            remoteExceptionToPolicyMap));
-    RetryPolicy methodPolicy = RetryPolicies.retryByException(
-        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
-    Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
-
-    methodNameToPolicyMap.put("create", methodPolicy);
-
-    return (ClientNamenodeWireProtocol) RetryProxy.create(
-        ClientNamenodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
-  }
-
-  public ClientNamenodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
-      Configuration conf, UserGroupInformation ugi) throws IOException {
-    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
-  }
-
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocolName,
-      long clientVersion, int clientMethodHash)
-      throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocolName, clientVersion, clientMethodHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion) throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public LocatedBlocks getBlockLocations(String src, long offset, long length)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return LocatedBlocksWritable
-        .convertLocatedBlocks(rpcProxy.getBlockLocations(src, offset, length));
-  }
-
-  @Override
-  public FsServerDefaults getServerDefaults() throws IOException {
-    return FsServerDefaultsWritable
-        .convert(rpcProxy.getServerDefaults());
-  }
-
-  @Override
-  public void create(String src, FsPermission masked, String clientName,
-      EnumSetWritable<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize) throws AccessControlException,
-      AlreadyBeingCreatedException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.create(src, FsPermissionWritable.convertPermission(masked),
-        clientName, flag, createParent, replication, blockSize);
-
-  }
-
-  @Override
-  public LocatedBlock append(String src, String clientName)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable
-        .convertLocatedBlock(rpcProxy.append(src, clientName));
-  }
-
-  @Override
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return rpcProxy.setReplication(src, replication);
-  }
-
-  @Override
-  public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setPermission(src,
-        FsPermissionWritable.convertPermission(permission));
-
-  }
-
-  @Override
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setOwner(src, username, groupname);
-
-  }
-
-  @Override
-  public void abandonBlock(ExtendedBlock b, String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.abandonBlock(
-        ExtendedBlockWritable.convertExtendedBlock(b), src, holder);
-
-  }
-
-  @Override
-  public LocatedBlock addBlock(String src, String clientName,
-      ExtendedBlock previous, DatanodeInfo[] excludeNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable
-        .convertLocatedBlock(rpcProxy.addBlock(src, clientName,
-            ExtendedBlockWritable.convertExtendedBlock(previous),
-            DatanodeInfoWritable.convertDatanodeInfo(excludeNodes)));
-  }
-
-  @Override
-  public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
-      DatanodeInfo[] existings, DatanodeInfo[] excludes,
-      int numAdditionalNodes, String clientName) throws AccessControlException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    return LocatedBlockWritable
-        .convertLocatedBlock(rpcProxy.getAdditionalDatanode(src,
-            ExtendedBlockWritable.convertExtendedBlock(blk),
-            DatanodeInfoWritable.convertDatanodeInfo(existings),
-            DatanodeInfoWritable.convertDatanodeInfo(excludes),
-            numAdditionalNodes, clientName));
-  }
-
-  @Override
-  public boolean complete(String src, String clientName, ExtendedBlock last)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return rpcProxy.complete(src, clientName,
-        ExtendedBlockWritable.convertExtendedBlock(last));
-  }
-
-  @Override
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    rpcProxy.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
-
-  }
-
-  @Override
-  public boolean rename(String src, String dst) throws UnresolvedLinkException,
-      IOException {
-    return rpcProxy.rename(src, dst);
-  }
-
-  @Override
-  public void concat(String trg, String[] srcs) throws IOException,
-      UnresolvedLinkException {
-    rpcProxy.concat(trg, srcs);
-
-  }
-
-  @Override
-  public void rename2(String src, String dst, Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.rename2(src, dst, options);
-
-  }
-
-  @Override
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    return rpcProxy.delete(src, recursive);
-  }
-
-  @Override
-  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-
-    return rpcProxy.mkdirs(src,
-        FsPermissionWritable.convertPermission(masked), createParent);
-  }
-
-  @Override
-  public DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return DirectoryListingWritable.convertDirectoryListing(
-        rpcProxy.getListing(src, startAfter, needLocation));
-  }
-
-  @Override
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException {
-    rpcProxy.renewLease(clientName);
-
-  }
-
-  @Override
-  public boolean recoverLease(String src, String clientName) throws IOException {
-    return rpcProxy.recoverLease(src, clientName);
-  }
-
-  @Override
-  public long[] getStats() throws IOException {
-    return rpcProxy.getStats();
-  }
-
-  @Override
-  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
-      throws IOException {
-    return DatanodeInfoWritable.convertDatanodeInfo(
-        rpcProxy.getDatanodeReport(type));
-  }
-
-  @Override
-  public long getPreferredBlockSize(String filename) throws IOException,
-      UnresolvedLinkException {
-    return rpcProxy.getPreferredBlockSize(filename);
-  }
-
-  @Override
-  public boolean setSafeMode(SafeModeAction action) throws IOException {
-    return rpcProxy.setSafeMode(action);
-  }
-
-  @Override
-  public void saveNamespace() throws AccessControlException, IOException {
-    rpcProxy.saveNamespace();
-
-  }
-
-  @Override
-  public boolean restoreFailedStorage(String arg)
-      throws AccessControlException, IOException{
-    return rpcProxy.restoreFailedStorage(arg);
-  }
-
-  @Override
-  public void refreshNodes() throws IOException {
-    rpcProxy.refreshNodes();
-
-  }
-
-  @Override
-  public void finalizeUpgrade() throws IOException {
-    rpcProxy.finalizeUpgrade();
-
-  }
-
-  @Override
-  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
-      throws IOException {
-    return UpgradeStatusReportWritable.convert(
-        rpcProxy.distributedUpgradeProgress(action));
-  }
-
-  @Override
-  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
-      throws IOException {
-    return CorruptFileBlocksWritable.convertCorruptFileBlocks(
-        rpcProxy.listCorruptFileBlocks(path, cookie));
-  }
-
-  @Override
-  public void metaSave(String filename) throws IOException {
-    rpcProxy.metaSave(filename);
-
-  }
-
-  @Override
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable.convertHdfsFileStatus(
-        rpcProxy.getFileInfo(src));
-  }
-
-  @Override
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException {
-    return HdfsFileStatusWritable
-        .convertHdfsFileStatus(rpcProxy.getFileLinkInfo(src));
-  }
-
-  @Override
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    return ContentSummaryWritable
-        .convert(rpcProxy.getContentSummary(path));
-  }
-
-  @Override
-  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setQuota(path, namespaceQuota, diskspaceQuota);
-
-  }
-
-  @Override
-  public void fsync(String src, String client) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    rpcProxy.fsync(src, client);
-
-  }
-
-  @Override
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    rpcProxy.setTimes(src, mtime, atime);
-
-  }
-
-  @Override
-  public void createSymlink(String target, String link, FsPermission dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    rpcProxy.createSymlink(target, link,
-        FsPermissionWritable.convertPermission(dirPerm), createParent);
-
-  }
-
-  @Override
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException {
-    return rpcProxy.getLinkTarget(path);
-  }
-
-  @Override
-  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
-      String clientName) throws IOException {
-    return LocatedBlockWritable.convertLocatedBlock(
-        rpcProxy.updateBlockForPipeline(
-            ExtendedBlockWritable.convertExtendedBlock(block), clientName));
-  }
-
-  @Override
-  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException {
-    rpcProxy.updatePipeline(clientName,
-        ExtendedBlockWritable.convertExtendedBlock(oldBlock),
-        ExtendedBlockWritable.convertExtendedBlock(newBlock),
-        DatanodeIDWritable.convertDatanodeID(newNodes));
-
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
-      throws IOException {
-    return rpcProxy.getDelegationToken(renewer);
-  }
-
-  @Override
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    return rpcProxy.renewDelegationToken(token);
-  }
-
-  @Override
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    rpcProxy.cancelDelegationToken(token);
-  }
-
-  @Override
-  public void setBalancerBandwidth(long bandwidth) throws IOException {
-    rpcProxy.setBalancerBandwidth(bandwidth);
-  }
-}

+ 0 - 478
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ClientNamenodeWireProtocol.java

@@ -1,478 +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.protocolR23Compatible;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.ipc.ProtocolInfo;
-
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
-
-/**********************************************************************
- * This class defines the actual protocol used to communicate with the
- * NN via RPC using writable types.
- * The parameters in the methods which are specified in the
- * package are separate from those used internally in the NN and DFSClient
- * and hence need to be converted using {@link ClientNamenodeProtocolTranslatorR23}
- * and {@link ClientNamenodeProtocolServerSideTranslatorR23}.
- *
- **********************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
-@TokenInfo(DelegationTokenSelector.class)
-@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME)
-public interface ClientNamenodeWireProtocol extends VersionedProtocol {
-
-  /**
-   * Changes to the protocol:
-   * 
-   * Do NOT change a method's signature (ie name, parameters, parameter types
-   * or exceptions thrown). If you need to make changes then ADD new methods and
-   * new data types.
-   * Hence if you maintain compatibility you will NOT have to change
-   * the version number below. The version number is changed ONLY
-   * if you break compatibility (which is a big deal).
-   * Hence the version number is really a Major Version Number.
-   *
-   * The log of historical changes prior to 69 can be retrieved from the svn.
-   * ALL changes since version 69L are recorded.
-   * Version number is changed ONLY for Incompatible changes.
-   *  (note previously we used to change version number for both
-   *  compatible and incompatible changes).
-   * 69: Eliminate overloaded method names. (Compatible)
-   * 70: Separation of Datatypes - the client namenode protocol is implemented
-   *     in this class instead of in 
-   *           {@link org.apache.hadoop.hdfs.protocol.ClientProtocol}
-   *     as was done prior to version 70.
-   */
-  public static final long versionID = 70L;
-  
-  ///////////////////////////////////////
-  // File contents
-  ///////////////////////////////////////
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getBlockLocations}
-   */
-  public LocatedBlocksWritable getBlockLocations(String src,
-                                         long offset,
-                                         long length) 
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getServerDefaults()}
-   */
-  public FsServerDefaultsWritable getServerDefaults() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#create(String, 
-   * org.apache.hadoop.fs.permission.FsPermission, String, 
-   * EnumSetWritable, boolean, short, long)}
-   */
-  public void create(String src, FsPermissionWritable masked, String clientName,
-      EnumSetWritable<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize) throws AccessControlException,
-      AlreadyBeingCreatedException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#append(String, String)}
-   */
-  public LocatedBlockWritable append(String src, String clientName)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setReplication(String, short)}
-   */
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setPermission(String,
-   * org.apache.hadoop.fs.permission.FsPermission)}
-   */
-  public void setPermission(String src, FsPermissionWritable permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setOwner(String, String, String)}
-   */
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#abandonBlock(
-   * org.apache.hadoop.hdfs.protocol.ExtendedBlock, String, String)}
-   */
-  public void abandonBlock(ExtendedBlockWritable b, String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#addBlock(String, 
-   * String, org.apache.hadoop.hdfs.protocol.ExtendedBlock, 
-   * org.apache.hadoop.hdfs.protocol.DatanodeInfo[])}
-   */
-  public LocatedBlockWritable addBlock(String src, String clientName,
-      ExtendedBlockWritable previous, DatanodeInfoWritable[] excludeNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getAdditionalDatanode}
-   */
-  public LocatedBlockWritable getAdditionalDatanode(
-      final String src, final ExtendedBlockWritable blk,
-      final DatanodeInfoWritable[] existings,
-      final DatanodeInfoWritable[] excludes,
-      final int numAdditionalNodes, final String clientName
-      ) throws AccessControlException, FileNotFoundException,
-          SafeModeException, UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#complete}
-   */
-  public boolean complete(
-      String src, String clientName, ExtendedBlockWritable last)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#reportBadBlocks}
-   */
-  public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException;
-
-  ///////////////////////////////////////
-  // Namespace management
-  ///////////////////////////////////////
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#rename(String, String)}
-   */
-  public boolean rename(String src, String dst) 
-      throws UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#concat(String, String[])}
-   */
-  public void concat(String trg, String[] srcs) 
-      throws IOException, UnresolvedLinkException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#rename2}
-   */
-  public void rename2(String src, String dst, Options.Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#delete(String, boolean)}
-   */
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#mkdirs}
-   */
-  public boolean mkdirs(
-      String src, FsPermissionWritable masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getListing}
-   */
-  public DirectoryListingWritable getListing(String src,
-                                     byte[] startAfter,
-                                     boolean needLocation)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  ///////////////////////////////////////
-  // System issues and management
-  ///////////////////////////////////////
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#renewLease(String)}
-   */
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#recoverLease(String, String)}
-   */
-  public boolean recoverLease(String src, String clientName) throws IOException;
-
-  public int GET_STATS_CAPACITY_IDX = 0;
-  public int GET_STATS_USED_IDX = 1;
-  public int GET_STATS_REMAINING_IDX = 2;
-  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
-  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
-  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getStats()}
-   */
-  public long[] getStats() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getDatanodeReport}
-   */
-  public DatanodeInfoWritable[] getDatanodeReport(
-      HdfsConstants.DatanodeReportType type)
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getPreferredBlockSize}
-   */
-  public long getPreferredBlockSize(String filename) 
-      throws IOException, UnresolvedLinkException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction)}
-   */
-  public boolean setSafeMode(HdfsConstants.SafeModeAction action) 
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()}
-   */
-  public void saveNamespace() throws AccessControlException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String)}
-   */
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#refreshNodes()}
-   */
-  public void refreshNodes() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#finalizeUpgrade()}
-   */
-  public void finalizeUpgrade() throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#distributedUpgradeProgress}
-   */
-  public UpgradeStatusReportWritable distributedUpgradeProgress(
-      UpgradeAction action) 
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#listCorruptFileBlocks(String, String)}
-   */
-  public CorruptFileBlocksWritable
-    listCorruptFileBlocks(String path, String cookie)
-    throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#metaSave(String)}
-   */
-  public void metaSave(String filename) throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setBalancerBandwidth(long)}
-   */
-  public void setBalancerBandwidth(long bandwidth) throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getFileInfo(String)}
-   */
-  public HdfsFileStatusWritable getFileInfo(String src)
-      throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getFileLinkInfo(String)}
-   */
-  public HdfsFileStatusWritable getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getContentSummary(String)}
-   */
-  public ContentSummaryWritable getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long)}
-   */
-  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#fsync(String, String)}
-   */
-  public void fsync(String src, String client) 
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setTimes(String, long, long)}
-   */
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#createSymlink}
-   */
-  public void createSymlink(
-      String target, String link, FsPermissionWritable dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getLinkTarget(String)}
-   */
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException; 
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#updateBlockForPipeline}
-   */
-  public LocatedBlockWritable updateBlockForPipeline(
-      ExtendedBlockWritable block, String clientName) throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#updatePipeline}
-   */
-  public void updatePipeline(String clientName, ExtendedBlockWritable oldBlock, 
-      ExtendedBlockWritable newBlock, DatanodeIDWritable[] newNodes)
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getDelegationToken(Text)}
-   */
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
-      throws IOException;
-
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#renewDelegationToken(Token)}
-   */
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException;
-  
-  /**
-   * The specification of this method matches that of
-   * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#cancelDelegationToken(Token)}
-   */
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException;
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
-           getProtocolSignature2(String protocol, 
-      long clientVersion,
-      int clientMethodsHash) throws IOException;
-}

+ 0 - 184
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ContentSummaryWritable.java

@@ -1,184 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-
-/** Store the summary of a content (a directory or a file). */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class ContentSummaryWritable implements Writable{
-  private long length;
-  private long fileCount;
-  private long directoryCount;
-  private long quota;
-  private long spaceConsumed;
-  private long spaceQuota;
-  
-  
-  public static org.apache.hadoop.fs.ContentSummary convert(ContentSummaryWritable cs) {
-    if (cs == null) return null;
-    return new org.apache.hadoop.fs.ContentSummary(
-      cs.getLength(), cs.getFileCount(), cs.getDirectoryCount(), cs.getQuota(),
-      cs.getSpaceConsumed(), cs.getSpaceQuota());
-  }
-  
-  public static ContentSummaryWritable convert(org.apache.hadoop.fs.ContentSummary cs) {
-    if (cs == null) return null;
-    return new  ContentSummaryWritable(
-      cs.getLength(), cs.getFileCount(), cs.getDirectoryCount(), cs.getQuota(),
-      cs.getSpaceConsumed(), cs.getSpaceQuota());
-  }
-
-  /** Constructor */
-  public ContentSummaryWritable() {}
-  
-  /** Constructor */
-  public ContentSummaryWritable(long length, long fileCount, long directoryCount) {
-    this(length, fileCount, directoryCount, -1L, length, -1L);
-  }
-
-  /** Constructor */
-  public ContentSummaryWritable(
-      long length, long fileCount, long directoryCount, long quota,
-      long spaceConsumed, long spaceQuota) {
-    this.length = length;
-    this.fileCount = fileCount;
-    this.directoryCount = directoryCount;
-    this.quota = quota;
-    this.spaceConsumed = spaceConsumed;
-    this.spaceQuota = spaceQuota;
-  }
-
-  /** @return the length */
-  public long getLength() {return length;}
-
-  /** @return the directory count */
-  public long getDirectoryCount() {return directoryCount;}
-
-  /** @return the file count */
-  public long getFileCount() {return fileCount;}
-  
-  /** Return the directory quota */
-  public long getQuota() {return quota;}
-  
-  /** Retuns (disk) space consumed */ 
-  public long getSpaceConsumed() {return spaceConsumed;}
-
-  /** Returns (disk) space quota */
-  public long getSpaceQuota() {return spaceQuota;}
-  
-  @InterfaceAudience.Private
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(length);
-    out.writeLong(fileCount);
-    out.writeLong(directoryCount);
-    out.writeLong(quota);
-    out.writeLong(spaceConsumed);
-    out.writeLong(spaceQuota);
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.length = in.readLong();
-    this.fileCount = in.readLong();
-    this.directoryCount = in.readLong();
-    this.quota = in.readLong();
-    this.spaceConsumed = in.readLong();
-    this.spaceQuota = in.readLong();
-  }
-  
-  /** 
-   * Output format:
-   * <----12----> <----12----> <-------18------->
-   *    DIR_COUNT   FILE_COUNT       CONTENT_SIZE FILE_NAME    
-   */
-  private static final String STRING_FORMAT = "%12d %12d %18d ";
-  /** 
-   * Output format:
-   * <----12----> <----15----> <----15----> <----15----> <----12----> <----12----> <-------18------->
-   *    QUOTA   REMAINING_QUATA SPACE_QUOTA SPACE_QUOTA_REM DIR_COUNT   FILE_COUNT   CONTENT_SIZE     FILE_NAME    
-   */
-  private static final String QUOTA_STRING_FORMAT = "%12s %15s ";
-  private static final String SPACE_QUOTA_STRING_FORMAT = "%15s %15s ";
-  
-  /** The header string */
-  private static final String HEADER = String.format(
-      STRING_FORMAT.replace('d', 's'), "directories", "files", "bytes");
-
-  private static final String QUOTA_HEADER = String.format(
-      QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT, 
-      "quota", "remaining quota", "space quota", "reamaining quota") +
-      HEADER;
-  
-  /** Return the header of the output.
-   * if qOption is false, output directory count, file count, and content size;
-   * if qOption is true, output quota and remaining quota as well.
-   * 
-   * @param qOption a flag indicating if quota needs to be printed or not
-   * @return the header of the output
-   */
-  public static String getHeader(boolean qOption) {
-    return qOption ? QUOTA_HEADER : HEADER;
-  }
-  
-  @Override
-  public String toString() {
-    return toString(true);
-  }
-
-  /** Return the string representation of the object in the output format.
-   * if qOption is false, output directory count, file count, and content size;
-   * if qOption is true, output quota and remaining quota as well.
-   * 
-   * @param qOption a flag indicating if quota needs to be printed or not
-   * @return the string representation of the object
-   */
-  public String toString(boolean qOption) {
-    String prefix = "";
-    if (qOption) {
-      String quotaStr = "none";
-      String quotaRem = "inf";
-      String spaceQuotaStr = "none";
-      String spaceQuotaRem = "inf";
-      
-      if (quota>0) {
-        quotaStr = Long.toString(quota);
-        quotaRem = Long.toString(quota-(directoryCount+fileCount));
-      }
-      if (spaceQuota>0) {
-        spaceQuotaStr = Long.toString(spaceQuota);
-        spaceQuotaRem = Long.toString(spaceQuota - spaceConsumed);        
-      }
-      
-      prefix = String.format(QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT, 
-                             quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
-    }
-    
-    return prefix + String.format(STRING_FORMAT, directoryCount, 
-                                  fileCount, length);
-  }
-}

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CorruptFileBlocksWritable.java

@@ -1,88 +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.protocolR23Compatible;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.Text;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-/**
- * Contains a list of paths corresponding to corrupt files and a cookie
- * used for iterative calls to NameNode.listCorruptFileBlocks.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class CorruptFileBlocksWritable implements Writable {
-
-  private String[] files;
-  private String cookie;
-
-  static public org.apache.hadoop.hdfs.protocol.CorruptFileBlocks 
-    convertCorruptFileBlocks(CorruptFileBlocksWritable c) {
-    if (c == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.CorruptFileBlocks(
-        c.getFiles(), c.getCookie());
-  }
-  
-  public static CorruptFileBlocksWritable convertCorruptFilesBlocks(
-      org.apache.hadoop.hdfs.protocol.CorruptFileBlocks c) {
-    if (c == null) return null;
-    return new CorruptFileBlocksWritable(c.getFiles(), c.getCookie());
-  }
- 
-  public CorruptFileBlocksWritable() {
-    this(new String[0], "");
-  }
-
-  public CorruptFileBlocksWritable(String[] files, String cookie) {
-    this.files = files;
-    this.cookie = cookie;
-  }
- 
-  public String[] getFiles() {
-    return files;
-  }
-
-  public String getCookie() {
-    return cookie;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int fileCount = in.readInt();
-    files = new String[fileCount];
-    for (int i = 0; i < fileCount; i++) {
-      files[i] = Text.readString(in);
-    }
-    cookie = Text.readString(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(files.length);
-    for (int i = 0; i < files.length; i++) {
-      Text.writeString(out, files[i]);
-    }
-    Text.writeString(out, cookie);
-  }
-}

+ 0 - 209
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeIDWritable.java

@@ -1,209 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
-import org.apache.hadoop.io.Writable;
-
-/**
- * DatanodeID is composed of the data node 
- * name (hostname:portNumber) and the data storage ID, 
- * which it currently represents.
- * 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DatanodeIDWritable implements Writable {
-  public static final DatanodeIDWritable[] EMPTY_ARRAY = {}; 
-
-  public String name;      /// hostname:portNumber
-  public String storageID; /// unique per cluster storageID
-  protected int infoPort;     /// the port where the infoserver is running
-  public int ipcPort;     /// the port where the ipc server is running
-
-  
-  static public DatanodeIDWritable[] 
-      convertDatanodeID(org.apache.hadoop.hdfs.protocol.DatanodeID[] did) {
-    if (did == null) return null;
-    final int len = did.length;
-    DatanodeIDWritable[] result = new DatanodeIDWritable[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = convertDatanodeID(did[i]);
-    }
-    return result;
-  }
-  
-  static public org.apache.hadoop.hdfs.protocol.DatanodeID[] 
-      convertDatanodeID(DatanodeIDWritable[] did) {
-    if (did == null) return null;
-    final int len = did.length;
-    org.apache.hadoop.hdfs.protocol.DatanodeID[] result = new org.apache.hadoop.hdfs.protocol.DatanodeID[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = convertDatanodeID(did[i]);
-    }
-    return result;
-  }
-  
-  static public org.apache.hadoop.hdfs.protocol.DatanodeID convertDatanodeID(
-      DatanodeIDWritable did) {
-    if (did == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.DatanodeID(
-        did.getName(), did.getStorageID(), did.getInfoPort(), did.getIpcPort());
-    
-  }
-  
-  public static DatanodeIDWritable convertDatanodeID(org.apache.hadoop.hdfs.protocol.DatanodeID from) {
-    return new DatanodeIDWritable(from.getName(),
-        from.getStorageID(),
-        from.getInfoPort(),
-        from.getIpcPort());
-  }
-  
-  /** Equivalent to DatanodeID(""). */
-  public DatanodeIDWritable() {this("");}
-
-  /** Equivalent to DatanodeID(nodeName, "", -1, -1). */
-  public DatanodeIDWritable(String nodeName) {this(nodeName, "", -1, -1);}
-
-  /**
-   * DatanodeID copy constructor
-   * 
-   * @param from
-   */
-  public DatanodeIDWritable(DatanodeIDWritable from) {
-    this(from.getName(),
-        from.getStorageID(),
-        from.getInfoPort(),
-        from.getIpcPort());
-  }
-  
-  /**
-   * Create DatanodeID
-   * @param nodeName (hostname:portNumber) 
-   * @param storageID data storage ID
-   * @param infoPort info server port 
-   * @param ipcPort ipc server port
-   */
-  public DatanodeIDWritable(String nodeName, String storageID,
-      int infoPort, int ipcPort) {
-    this.name = nodeName;
-    this.storageID = storageID;
-    this.infoPort = infoPort;
-    this.ipcPort = ipcPort;
-  }
-  
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public void setInfoPort(int infoPort) {
-    this.infoPort = infoPort;
-  }
-  
-  public void setIpcPort(int ipcPort) {
-    this.ipcPort = ipcPort;
-  }
-  
-  /**
-   * @return hostname:portNumber.
-   */
-  public String getName() {
-    return name;
-  }
-  
-  /**
-   * @return data storage ID.
-   */
-  public String getStorageID() {
-    return this.storageID;
-  }
-
-  /**
-   * @return infoPort (the port at which the HTTP server bound to)
-   */
-  public int getInfoPort() {
-    return infoPort;
-  }
-
-  /**
-   * @return ipcPort (the port at which the IPC server bound to)
-   */
-  public int getIpcPort() {
-    return ipcPort;
-  }
-
-  /**
-   * sets the data storage ID.
-   */
-  public void setStorageID(String storageID) {
-    this.storageID = storageID;
-  }
-
-  /**
-   * @return hostname and no :portNumber.
-   */
-  public String getHost() {
-    int colon = name.indexOf(":");
-    if (colon < 0) {
-      return name;
-    } else {
-      return name.substring(0, colon);
-    }
-  }
-  
-  public int getPort() {
-    int colon = name.indexOf(":");
-    if (colon < 0) {
-      return 50010; // default port.
-    }
-    return Integer.parseInt(name.substring(colon+1));
-  }
-
-  
-  public String toString() {
-    return name;
-  }    
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    DeprecatedUTF8.writeString(out, name);
-    DeprecatedUTF8.writeString(out, storageID);
-    out.writeShort(infoPort);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    name = DeprecatedUTF8.readString(in);
-    storageID = DeprecatedUTF8.readString(in);
-    // the infoPort read could be negative, if the port is a large number (more
-    // than 15 bits in storage size (but less than 16 bits).
-    // So chop off the first two bytes (and hence the signed bits) before 
-    // setting the field.
-    this.infoPort = in.readShort() & 0x0000ffff;
-  }
-}

+ 0 - 334
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeInfoWritable.java

@@ -1,334 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.NodeBase;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-
-/** 
- * DatanodeInfo represents the status of a DataNode.
- * This object is used for communication in the
- * Datanode Protocol and the Client Protocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DatanodeInfoWritable extends DatanodeIDWritable  {
-  protected long capacity;
-  protected long dfsUsed;
-  protected long remaining;
-  protected long blockPoolUsed;
-  protected long lastUpdate;
-  protected int xceiverCount;
-  protected String location = NetworkTopology.DEFAULT_RACK;
-
-  /** HostName as supplied by the datanode during registration as its 
-   * name. Namenode uses datanode IP address as the name.
-   */
-  protected String hostName = null;
-  
-  // administrative states of a datanode
-  public enum AdminStates {
-    NORMAL(DatanodeInfo.AdminStates.NORMAL.toString()), 
-    DECOMMISSION_INPROGRESS(DatanodeInfo.AdminStates.DECOMMISSION_INPROGRESS.toString()), 
-    DECOMMISSIONED(DatanodeInfo.AdminStates.DECOMMISSIONED.toString());
-
-    final String value;
-
-    AdminStates(final String v) {
-      this.value = v;
-    }
-
-    public String toString() {
-      return value;
-    }
-    
-    public static AdminStates fromValue(final String value) {
-      for (AdminStates as : AdminStates.values()) {
-        if (as.value.equals(value)) return as;
-      }
-      throw new HadoopIllegalArgumentException("Unknown Admin State" + value);
-    }
-  }
-
-  protected AdminStates adminState;
-  
-  static public DatanodeInfo convertDatanodeInfo(DatanodeInfoWritable di) {
-    if (di == null) return null;
-    return new DatanodeInfo(
-        new org.apache.hadoop.hdfs.protocol.DatanodeID(di.getName(), di.getStorageID(), di.getInfoPort(), di.getIpcPort()),
-        di.getNetworkLocation(), di.getHostName(),
-         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
-        di.getBlockPoolUsed()  ,  di.getLastUpdate() , di.getXceiverCount() ,
-        DatanodeInfo.AdminStates.fromValue(di.getAdminState().value)); 
-  }
-  
-  
-  static public DatanodeInfo[] convertDatanodeInfo(DatanodeInfoWritable di[]) {
-    if (di == null) return null;
-    DatanodeInfo[] result = new DatanodeInfo[di.length];
-    for (int i = 0; i < di.length; i++) {
-      result[i] = convertDatanodeInfo(di[i]);
-    }    
-    return result;
-  }
-  
-  static public DatanodeInfoWritable[] convertDatanodeInfo(DatanodeInfo[] di) {
-    if (di == null) return null;
-    DatanodeInfoWritable[] result = new DatanodeInfoWritable[di.length];
-    for (int i = 0; i < di.length; i++) {
-      result[i] = new DatanodeInfoWritable(new DatanodeIDWritable(di[i].getName(), di[i].getStorageID(), di[i].getInfoPort(), di[i].getIpcPort()),
-          di[i].getNetworkLocation(), di[i].getHostName(),
-          di[i].getCapacity(),  di[i].getDfsUsed(),  di[i].getRemaining(),
-          di[i].getBlockPoolUsed()  ,  di[i].getLastUpdate() , di[i].getXceiverCount() ,
-          AdminStates.fromValue(di[i].getAdminState().toString()));
-    }    
-    return result;
-  }
-  
-  static public DatanodeInfoWritable convertDatanodeInfo(DatanodeInfo di) {
-    if (di == null) return null;
-    return new DatanodeInfoWritable(new DatanodeIDWritable(di.getName(),
-        di.getStorageID(), di.getInfoPort(), di.getIpcPort()),
-        di.getNetworkLocation(), di.getHostName(), di.getCapacity(),
-        di.getDfsUsed(), di.getRemaining(), di.getBlockPoolUsed(),
-        di.getLastUpdate(), di.getXceiverCount(), 
-        AdminStates.fromValue(di.getAdminState().toString()));
-  }
-
-  public DatanodeInfoWritable() {
-    super();
-    adminState = null;
-  }
-  
-  public DatanodeInfoWritable(DatanodeInfoWritable from) {
-    super(from);
-    this.capacity = from.getCapacity();
-    this.dfsUsed = from.getDfsUsed();
-    this.remaining = from.getRemaining();
-    this.blockPoolUsed = from.getBlockPoolUsed();
-    this.lastUpdate = from.getLastUpdate();
-    this.xceiverCount = from.getXceiverCount();
-    this.location = from.getNetworkLocation();
-    this.adminState = from.adminState;
-    this.hostName = from.hostName;
-  }
-
-  public DatanodeInfoWritable(DatanodeIDWritable nodeID) {
-    super(nodeID);
-    this.capacity = 0L;
-    this.dfsUsed = 0L;
-    this.remaining = 0L;
-    this.blockPoolUsed = 0L;
-    this.lastUpdate = 0L;
-    this.xceiverCount = 0;
-    this.adminState = null;    
-  }
-  
-  protected DatanodeInfoWritable(DatanodeIDWritable nodeID, String location, String hostName) {
-    this(nodeID);
-    this.location = location;
-    this.hostName = hostName;
-  }
-  
-  public DatanodeInfoWritable(DatanodeIDWritable nodeID, String location, String hostName,
-      final long capacity, final long dfsUsed, final long remaining,
-      final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
-      final AdminStates adminState) {
-    this(nodeID, location, hostName);
-    this.capacity = capacity;
-    this.dfsUsed = dfsUsed;
-    this.remaining = remaining;
-    this.blockPoolUsed = blockPoolUsed;
-    this.lastUpdate = lastUpdate;
-    this.xceiverCount = xceiverCount;
-    this.adminState = adminState;
-  }
-  
-  /** The raw capacity. */
-  public long getCapacity() { return capacity; }
-  
-  /** The used space by the data node. */
-  public long getDfsUsed() { return dfsUsed; }
-
-  /** The used space by the block pool on data node. */
-  public long getBlockPoolUsed() { return blockPoolUsed; }
-
-  /** The used space by the data node. */
-  public long getNonDfsUsed() { 
-    long nonDFSUsed = capacity - dfsUsed - remaining;
-    return nonDFSUsed < 0 ? 0 : nonDFSUsed;
-  }
-
-  /** The used space by the data node as percentage of present capacity */
-  public float getDfsUsedPercent() { 
-    return DFSUtil.getPercentUsed(dfsUsed, capacity);
-  }
-
-  /** The raw free space. */
-  public long getRemaining() { return remaining; }
-
-  /** Used space by the block pool as percentage of present capacity */
-  public float getBlockPoolUsedPercent() {
-    return DFSUtil.getPercentUsed(blockPoolUsed, capacity);
-  }
-  
-  /** The remaining space as percentage of configured capacity. */
-  public float getRemainingPercent() { 
-    return DFSUtil.getPercentRemaining(remaining, capacity);
-  }
-
-  /** The time when this information was accurate. */
-  public long getLastUpdate() { return lastUpdate; }
-
-  /** number of active connections */
-  public int getXceiverCount() { return xceiverCount; }
-
-  /** Sets raw capacity. */
-  public void setCapacity(long capacity) { 
-    this.capacity = capacity; 
-  }
-  
-  /** Sets the used space for the datanode. */
-  public void setDfsUsed(long dfsUsed) {
-    this.dfsUsed = dfsUsed;
-  }
-
-  /** Sets raw free space. */
-  public void setRemaining(long remaining) { 
-    this.remaining = remaining; 
-  }
-
-  /** Sets block pool used space */
-  public void setBlockPoolUsed(long bpUsed) { 
-    this.blockPoolUsed = bpUsed; 
-  }
-
-  /** Sets time when this information was accurate. */
-  public void setLastUpdate(long lastUpdate) { 
-    this.lastUpdate = lastUpdate; 
-  }
-
-  /** Sets number of active connections */
-  public void setXceiverCount(int xceiverCount) { 
-    this.xceiverCount = xceiverCount; 
-  }
-
-  /** rack name */
-  public String getNetworkLocation() {return location;}
-    
-  /** Sets the rack name */
-  public void setNetworkLocation(String location) {
-    this.location = NodeBase.normalize(location);
-  }
-  
-  public String getHostName() {
-    return (hostName == null || hostName.length()==0) ? getHost() : hostName;
-  }
-  
-  public void setHostName(String host) {
-    hostName = host;
-  }
-
-  /**
-   * Retrieves the admin state of this node.
-   */
-  public AdminStates getAdminState() {
-    if (adminState == null) {
-      return AdminStates.NORMAL;
-    }
-    return adminState;
-  }
-
-  /**
-   * Sets the admin state of this node.
-   */
-  protected void setAdminState(AdminStates newState) {
-    if (newState == AdminStates.NORMAL) {
-      adminState = null;
-    }
-    else {
-      adminState = newState;
-    }
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (DatanodeInfoWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new DatanodeInfoWritable(); }
-       });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-
-    out.writeShort(ipcPort);
-
-    out.writeLong(capacity);
-    out.writeLong(dfsUsed);
-    out.writeLong(remaining);
-    out.writeLong(blockPoolUsed);
-    out.writeLong(lastUpdate);
-    out.writeInt(xceiverCount);
-    Text.writeString(out, location);
-    Text.writeString(out, hostName == null? "" : hostName);
-    WritableUtils.writeEnum(out, getAdminState());
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-
-    this.ipcPort = in.readShort() & 0x0000ffff;
-
-    this.capacity = in.readLong();
-    this.dfsUsed = in.readLong();
-    this.remaining = in.readLong();
-    this.blockPoolUsed = in.readLong();
-    this.lastUpdate = in.readLong();
-    this.xceiverCount = in.readInt();
-    this.location = Text.readString(in);
-    this.hostName = Text.readString(in);
-    setAdminState(WritableUtils.readEnum(in, AdminStates.class));
-  }
-
-  /** Read a DatanodeInfo */
-  public static DatanodeInfoWritable read(DataInput in) throws IOException {
-    final DatanodeInfoWritable d = new DatanodeInfoWritable();
-    d.readFields(in);
-    return d;
-  }
-}

+ 0 - 157
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DirectoryListingWritable.java

@@ -1,157 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * This class defines a partial listing of a directory to support
- * iterative directory listing.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class DirectoryListingWritable implements Writable {
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (DirectoryListingWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new DirectoryListingWritable(); }
-       });
-  }
-
-  private HdfsFileStatusWritable[] partialListing;
-  private int remainingEntries;
-  
-  public static org.apache.hadoop.hdfs.protocol.DirectoryListing 
-    convertDirectoryListing(DirectoryListingWritable dl) {
-    if (dl == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.DirectoryListing(
-        HdfsFileStatusWritable.convertHdfsFileStatus(
-            dl.getPartialListing()), dl.getRemainingEntries());
-  }
-
-  public static DirectoryListingWritable convertDirectoryListing(
-      org.apache.hadoop.hdfs.protocol.DirectoryListing d) {
-    if (d == null) return null;
-    return new DirectoryListingWritable(
-        org.apache.hadoop.hdfs.protocolR23Compatible.HdfsFileStatusWritable.
-        convertHdfsFileStatus(d.getPartialListing()), d.getRemainingEntries());
-  } 
-  
-  /**
-   * default constructor
-   */
-  public DirectoryListingWritable() {
-  }
-  
-  /**
-   * constructor
-   * @param partialListing a partial listing of a directory
-   * @param remainingEntries number of entries that are left to be listed
-   */
-  public DirectoryListingWritable(HdfsFileStatusWritable[] partialListing, 
-      int remainingEntries) {
-    if (partialListing == null) {
-      throw new IllegalArgumentException("partial listing should not be null");
-    }
-    if (partialListing.length == 0 && remainingEntries != 0) {
-      throw new IllegalArgumentException("Partial listing is empty but " +
-          "the number of remaining entries is not zero");
-    }
-    this.partialListing = partialListing;
-    this.remainingEntries = remainingEntries;
-  }
-
-  /**
-   * Get the partial listing of file status
-   * @return the partial listing of file status
-   */
-  public HdfsFileStatusWritable[] getPartialListing() {
-    return partialListing;
-  }
-  
-  /**
-   * Get the number of remaining entries that are left to be listed
-   * @return the number of remaining entries that are left to be listed
-   */
-  public int getRemainingEntries() {
-    return remainingEntries;
-  }
-  
-  /**
-   * Check if there are more entries that are left to be listed
-   * @return true if there are more entries that are left to be listed;
-   *         return false otherwise.
-   */
-  public boolean hasMore() {
-    return remainingEntries != 0;
-  }
-  
-  /**
-   * Get the last name in this list
-   * @return the last name in the list if it is not empty; otherwise return null
-   */
-  public byte[] getLastName() {
-    if (partialListing.length == 0) {
-      return null;
-    }
-    return partialListing[partialListing.length-1].getLocalNameInBytes();
-  }
-
-  // Writable interface
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int numEntries = in.readInt();
-    partialListing = new HdfsFileStatusWritable[numEntries];
-    if (numEntries !=0 ) {
-      boolean hasLocation = in.readBoolean();
-      for (int i=0; i<numEntries; i++) {
-        if (hasLocation) {
-          partialListing[i] = new HdfsLocatedFileStatusWritable();
-        } else {
-          partialListing[i] = new HdfsFileStatusWritable();
-        }
-        partialListing[i].readFields(in);
-      }
-    }
-    remainingEntries = in.readInt();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(partialListing.length);
-    if (partialListing.length != 0) { 
-       if (partialListing[0] instanceof HdfsLocatedFileStatusWritable) {
-         out.writeBoolean(true);
-       } else {
-         out.writeBoolean(false);
-       }
-       for (HdfsFileStatusWritable fileStatus : partialListing) {
-         fileStatus.write(out);
-       }
-    }
-    out.writeInt(remainingEntries);
-  }
-}

+ 0 - 104
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExportedBlockKeysWritable.java

@@ -1,104 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.security.token.block.BlockKey;
-import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Object for passing block keys
- */
-@InterfaceAudience.Private
-public class ExportedBlockKeysWritable implements Writable {
-  private boolean isBlockTokenEnabled;
-  private long keyUpdateInterval;
-  private long tokenLifetime;
-  private BlockKey currentKey;
-  private BlockKey[] allKeys;
-
-  public ExportedBlockKeysWritable() {
-    this(false, 0, 0, new BlockKey(), new BlockKey[0]);
-  }
-
-  ExportedBlockKeysWritable(boolean isBlockTokenEnabled, long keyUpdateInterval,
-      long tokenLifetime, BlockKey currentKey, BlockKey[] allKeys) {
-    this.isBlockTokenEnabled = isBlockTokenEnabled;
-    this.keyUpdateInterval = keyUpdateInterval;
-    this.tokenLifetime = tokenLifetime;
-    this.currentKey = currentKey == null ? new BlockKey() : currentKey;
-    this.allKeys = allKeys == null ? new BlockKey[0] : allKeys;
-  }
-
-  // ///////////////////////////////////////////////
-  // Writable
-  // ///////////////////////////////////////////////
-  static { // register a ctor
-    WritableFactories.setFactory(ExportedBlockKeysWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new ExportedBlockKeysWritable();
-          }
-        });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeBoolean(isBlockTokenEnabled);
-    out.writeLong(keyUpdateInterval);
-    out.writeLong(tokenLifetime);
-    currentKey.write(out);
-    out.writeInt(allKeys.length);
-    for (int i = 0; i < allKeys.length; i++) {
-      allKeys[i].write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    isBlockTokenEnabled = in.readBoolean();
-    keyUpdateInterval = in.readLong();
-    tokenLifetime = in.readLong();
-    currentKey.readFields(in);
-    this.allKeys = new BlockKey[in.readInt()];
-    for (int i = 0; i < allKeys.length; i++) {
-      allKeys[i] = new BlockKey();
-      allKeys[i].readFields(in);
-    }
-  }
-
-  public static ExportedBlockKeysWritable convert(ExportedBlockKeys blockKeys) {
-    if (blockKeys == null) return null;
-    return new ExportedBlockKeysWritable(blockKeys.isBlockTokenEnabled(),
-        blockKeys.getKeyUpdateInterval(), blockKeys.getTokenLifetime(),
-        blockKeys.getCurrentKey(), blockKeys.getAllKeys());
-  }
-  
-  public ExportedBlockKeys convert() {
-    return new ExportedBlockKeys(isBlockTokenEnabled, keyUpdateInterval,
-        tokenLifetime, currentKey, allKeys);
-  }
-}

+ 0 - 121
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExtendedBlockWritable.java

@@ -1,121 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Identifies a Block uniquely across the block pools
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ExtendedBlockWritable implements Writable {
-  private String poolId;
-  private long blockId;
-  private long numBytes;
-  private long generationStamp;
-
-  static { // register a ctor
-    WritableFactories.setFactory(ExtendedBlockWritable.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new ExtendedBlockWritable();
-      }
-    });
-  }
-
-  static public org.apache.hadoop.hdfs.protocol.ExtendedBlock convertExtendedBlock(ExtendedBlockWritable eb) {
-    if (eb == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.ExtendedBlock( eb.getBlockPoolId(),  eb.getBlockId(),   eb.getNumBytes(),
-       eb.getGenerationStamp());
-  }
-  
-  public static ExtendedBlockWritable convertExtendedBlock(final org.apache.hadoop.hdfs.protocol.ExtendedBlock b) {
-    if (b == null) return null;
-    return new ExtendedBlockWritable(b.getBlockPoolId(), 
-        b.getBlockId(), b.getNumBytes(), b.getGenerationStamp());
-  }
-  
-  public ExtendedBlockWritable() {
-    this(null, 0, 0, 0);
-  }
-
-  public ExtendedBlockWritable(final ExtendedBlockWritable b) {
-    this(b.poolId, b.blockId, b.numBytes, b.generationStamp);
-  }
-  
-  public ExtendedBlockWritable(final String poolId, final long blockId) {
-    this(poolId, blockId, 0, 0);
-  }
-
-  public ExtendedBlockWritable(final String poolId, final long blkid, final long len,
-      final long genstamp) {
-    this.poolId = poolId;
-    this.blockId = blkid;
-    this.numBytes = len;
-    this.generationStamp = genstamp;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    DeprecatedUTF8.writeString(out, poolId);
-    out.writeLong(blockId);
-    out.writeLong(numBytes);
-    out.writeLong(generationStamp);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.poolId = DeprecatedUTF8.readString(in);
-    this.blockId = in.readLong();
-    this.numBytes = in.readLong();
-    this.generationStamp = in.readLong();
-    if (numBytes < 0) {
-      throw new IOException("Unexpected block size: " + numBytes);
-    }
-  }
-
-  public String getBlockPoolId() {
-    return poolId;
-  }
-
-  public long getNumBytes() {
-    return numBytes;
-  }
-
-  public long getBlockId() {
-    return blockId;
-  }
-
-  public long getGenerationStamp() {
-    return generationStamp;
-  }
-  
-  @Override // Object
-  public String toString() {
-    return poolId + ":" + (new org.apache.hadoop.hdfs.protocol.Block(blockId, numBytes, generationStamp));
-  }
-}

+ 0 - 80
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsPermissionWritable.java

@@ -1,80 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class FsPermissionWritable  implements Writable {
-  static final WritableFactory FACTORY = new WritableFactory() {
-	public Writable newInstance() { return new FsPermissionWritable(); }
-  };
-  static {                                      // register a ctor
-    WritableFactories.setFactory(FsPermissionWritable.class, FACTORY);
-  }
-  //POSIX permission style
-  private short thePermissions = 0;
-  
-  public static FsPermissionWritable convertPermission(org.apache.hadoop.fs.permission.FsPermission p) {
-    if (p == null) return null;
-    return new FsPermissionWritable(p.toShort());
-  }
-  
-  public static org.apache.hadoop.fs.permission.FsPermission convertPermission(FsPermissionWritable p) {
-    if (p == null) return null;
-    return new org.apache.hadoop.fs.permission.FsPermission(p.thePermissions);
-  }
-  
-  public static FsPermissionWritable getDefault() {
-    return new FsPermissionWritable((short)00777);
-  }
-  
-  FsPermissionWritable() {
-  }
-	FsPermissionWritable(short p) {
-	  thePermissions = p;
-	}
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeShort(thePermissions);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    thePermissions = in.readShort();
-  }
-
-  /**
-   * Create and initialize a {@link FsPermissionWritable} from {@link DataInput}.
-   */
-  public static FsPermissionWritable read(DataInput in) throws IOException {
-    FsPermissionWritable p = new FsPermissionWritable();
-    p.readFields(in);
-    return p;
-  }
-}

+ 0 - 122
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/FsServerDefaultsWritable.java

@@ -1,122 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/****************************************************
- * Provides server default configuration values to clients.
- * 
- ****************************************************/
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class FsServerDefaultsWritable implements Writable {
-
-  static { // register a ctor
-    WritableFactories.setFactory(FsServerDefaultsWritable.class, new WritableFactory() {
-      public Writable newInstance() {
-        return new FsServerDefaultsWritable();
-      }
-    });
-  }
-
-  private long blockSize;
-  private int bytesPerChecksum;
-  private int writePacketSize;
-  private short replication;
-  private int fileBufferSize;
-  
-  public static org.apache.hadoop.fs.FsServerDefaults convert(
-      FsServerDefaultsWritable fs) {
-    if (fs == null) return null;
-    return new org.apache.hadoop.fs.FsServerDefaults(
-        fs.getBlockSize(), fs.getBytesPerChecksum(), 
-        fs.getWritePacketSize(), fs.getReplication(), fs.getFileBufferSize());
-  }
-  
-  public static FsServerDefaultsWritable convert(
-      org.apache.hadoop.fs.FsServerDefaults fs) {
-    if (fs == null) return null;
-    return new FsServerDefaultsWritable(
-        fs.getBlockSize(), fs.getBytesPerChecksum(), 
-        fs.getWritePacketSize(), fs.getReplication(), fs.getFileBufferSize());
-  }
-  
-  public FsServerDefaultsWritable() {
-  }
-
-  public FsServerDefaultsWritable(long blockSize, int bytesPerChecksum,
-      int writePacketSize, short replication, int fileBufferSize) {
-    this.blockSize = blockSize;
-    this.bytesPerChecksum = bytesPerChecksum;
-    this.writePacketSize = writePacketSize;
-    this.replication = replication;
-    this.fileBufferSize = fileBufferSize;
-  }
-
-  public long getBlockSize() {
-    return blockSize;
-  }
-
-  public int getBytesPerChecksum() {
-    return bytesPerChecksum;
-  }
-
-  public int getWritePacketSize() {
-    return writePacketSize;
-  }
-
-  public short getReplication() {
-    return replication;
-  }
-
-  public int getFileBufferSize() {
-    return fileBufferSize;
-  }
-
-  // /////////////////////////////////////////
-  // Writable
-  // /////////////////////////////////////////
-  @Override
-  @InterfaceAudience.Private
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(blockSize);
-    out.writeInt(bytesPerChecksum);
-    out.writeInt(writePacketSize);
-    out.writeShort(replication);
-    out.writeInt(fileBufferSize);
-  }
-
-  @Override
-  @InterfaceAudience.Private
-  public void readFields(DataInput in) throws IOException {
-    blockSize = in.readLong();
-    bytesPerChecksum = in.readInt();
-    writePacketSize = in.readInt();
-    replication = in.readShort();
-    fileBufferSize = in.readInt();
-  }
-}

+ 0 - 348
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsFileStatusWritable.java

@@ -1,348 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-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 org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/** Interface that represents the over the wire information for a file.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class HdfsFileStatusWritable implements Writable {
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (HdfsFileStatusWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new HdfsFileStatusWritable(); }
-       });
-  }
-
-  private byte[] path;  // local name of the inode that's encoded in java UTF8
-  private byte[] symlink; // symlink target encoded in java UTF8 or null
-  private long length;
-  private boolean isdir;
-  private short block_replication;
-  private long blocksize;
-  private long modification_time;
-  private long access_time;
-  private FsPermissionWritable permission;
-  private String owner;
-  private String group;
-  
-  public static final byte[] EMPTY_NAME = new byte[0];
-
-  static public org.apache.hadoop.hdfs.protocol.HdfsFileStatus
-    convertHdfsFileStatus(HdfsFileStatusWritable fs) {
-    if (fs == null) return null;
-    return new org.apache.hadoop.hdfs.protocol.HdfsFileStatus(fs.getLen(),
-        fs.isDir(), fs.getReplication(), fs.getBlockSize(),
-        fs.getModificationTime(), fs.getAccessTime(), 
-        FsPermissionWritable.convertPermission(fs.getPermission()),
-        fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
-        fs.getLocalNameInBytes());
-  }
-  
-  static public HdfsFileStatusWritable[] convertHdfsFileStatus(org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] fs) {
-    if (fs == null) return null;
-    final int len = fs.length;
-    HdfsFileStatusWritable[] result = new HdfsFileStatusWritable[len];
-    for (int i = 0; i < len; ++i) {
-      if (fs[i] instanceof org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus) {
-        result[i] = 
-            HdfsLocatedFileStatusWritable.convertLocatedHdfsFileStatus(
-                (org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus)fs[i]);
-      } else {
-        result[i] = HdfsFileStatusWritable.convertHdfsFileStatus(fs[i]);
-      }
-    }
-    return result;
-  }
-  
-
-  public static org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] convertHdfsFileStatus(
-      HdfsFileStatusWritable[] fs) {
-    if (fs == null) return null;
-    final int len = fs.length;
-    org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] result = 
-        new org.apache.hadoop.hdfs.protocol.HdfsFileStatus[len];
-    for (int i = 0; i < len; ++i) {
-      if (fs[i] instanceof HdfsLocatedFileStatusWritable) {
-        result[i] = 
-            HdfsLocatedFileStatusWritable.convertLocatedHdfsFileStatus((HdfsLocatedFileStatusWritable)fs[i]);
-      } else {
-        result[i] = convertHdfsFileStatus(fs[i]);
-      }
-    }
-    return result;
-  }
- 
-  public static HdfsFileStatusWritable convertHdfsFileStatus(org.apache.hadoop.hdfs.protocol.HdfsFileStatus fs) {
-    if (fs == null) return null;
-    return new HdfsFileStatusWritable(fs.getLen(), fs.isDir(), fs.getReplication(),
-       fs.getBlockSize(),  fs.getModificationTime(),  fs.getAccessTime(),
-       org.apache.hadoop.hdfs.protocolR23Compatible.FsPermissionWritable.
-         convertPermission(fs.getPermission()),
-       fs.getOwner(),  fs.getGroup(), 
-       fs.getSymlinkInBytes(), fs.getLocalNameInBytes());
-  }
-  
-  /**
-   * default constructor
-   */
-  public HdfsFileStatusWritable() { 
-    this(0, false, 0, 0, 0, 0, null, null, null, null, null); 
-  }
-  
-  /**
-   * Constructor
-   * @param length the number of bytes the file has
-   * @param isdir if the path is a directory
-   * @param block_replication the replication factor
-   * @param blocksize the block size
-   * @param modification_time modification time
-   * @param access_time access time
-   * @param permission permission
-   * @param owner the owner of the path
-   * @param group the group of the path
-   * @param path the local name in java UTF8 encoding the same as that in-memory
-   */
-  public HdfsFileStatusWritable(long length, boolean isdir, int block_replication,
-                    long blocksize, long modification_time, long access_time,
-                    FsPermissionWritable permission, String owner, String group, 
-                    byte[] symlink, byte[] path) {
-    this.length = length;
-    this.isdir = isdir;
-    this.block_replication = (short)block_replication;
-    this.blocksize = blocksize;
-    this.modification_time = modification_time;
-    this.access_time = access_time;
-    this.permission = (permission == null) ? FsPermissionWritable.getDefault() : permission;
-    this.owner = (owner == null) ? "" : owner;
-    this.group = (group == null) ? "" : group;
-    this.symlink = symlink;
-    this.path = path;
-  }
-
-  /**
-   * Get the length of this file, in bytes.
-   * @return the length of this file, in bytes.
-   */
-  final public long getLen() {
-    return length;
-  }
-
-  /**
-   * Is this a directory?
-   * @return true if this is a directory
-   */
-  final public boolean isDir() {
-    return isdir;
-  }
-
-  /**
-   * Is this a symbolic link?
-   * @return true if this is a symbolic link
-   */
-  public boolean isSymlink() {
-    return symlink != null;
-  }
-   
-  /**
-   * Get the block size of the file.
-   * @return the number of bytes
-   */
-  final public long getBlockSize() {
-    return blocksize;
-  }
-
-  /**
-   * Get the replication factor of a file.
-   * @return the replication factor of a file.
-   */
-  final public short getReplication() {
-    return block_replication;
-  }
-
-  /**
-   * Get the modification time of the file.
-   * @return the modification time of file in milliseconds since January 1, 1970 UTC.
-   */
-  final public long getModificationTime() {
-    return modification_time;
-  }
-
-  /**
-   * Get the access time of the file.
-   * @return the access time of file in milliseconds since January 1, 1970 UTC.
-   */
-  final public long getAccessTime() {
-    return access_time;
-  }
-
-  /**
-   * Get FsPermission associated with the file.
-   * @return permssion
-   */
-  final public FsPermissionWritable getPermission() {
-    return permission;
-  }
-  
-  /**
-   * Get the owner of the file.
-   * @return owner of the file
-   */
-  final public String getOwner() {
-    return owner;
-  }
-  
-  /**
-   * Get the group associated with the file.
-   * @return group for the file. 
-   */
-  final public String getGroup() {
-    return group;
-  }
-  
-  /**
-   * Check if the local name is empty
-   * @return true if the name is empty
-   */
-  final public boolean isEmptyLocalName() {
-    return path.length == 0;
-  }
-
-  /**
-   * Get the string representation of the local name
-   * @return the local name in string
-   */
-  final public String getLocalName() {
-    return DFSUtil.bytes2String(path);
-  }
-  
-  /**
-   * Get the Java UTF8 representation of the local name
-   * @return the local name in java UTF8
-   */
-  final public byte[] getLocalNameInBytes() {
-    return path;
-  }
-
-  /**
-   * Get the string representation of the full path name
-   * @param parent the parent path
-   * @return the full path in string
-   */
-  final public String getFullName(final String parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-    
-    StringBuilder fullName = new StringBuilder(parent);
-    if (!parent.endsWith(Path.SEPARATOR)) {
-      fullName.append(Path.SEPARATOR);
-    }
-    fullName.append(getLocalName());
-    return fullName.toString();
-  }
-
-  /**
-   * Get the full path
-   * @param parent the parent path
-   * @return the full path
-   */
-  final public Path getFullPath(final Path parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-    
-    return new Path(parent, getLocalName());
-  }
-
-  /**
-   * Get the string representation of the symlink.
-   * @return the symlink as a string.
-   */
-  final public String getSymlink() {
-    return DFSUtil.bytes2String(symlink);
-  }
-  
-  final public byte[] getSymlinkInBytes() {
-    return symlink;
-  }
-  
-
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(path.length);
-    out.write(path);
-    out.writeLong(length);
-    out.writeBoolean(isdir);
-    out.writeShort(block_replication);
-    out.writeLong(blocksize);
-    out.writeLong(modification_time);
-    out.writeLong(access_time);
-    permission.write(out);
-    Text.writeString(out, owner);
-    Text.writeString(out, group);
-    out.writeBoolean(isSymlink());
-    if (isSymlink()) {
-      out.writeInt(symlink.length);
-      out.write(symlink);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int numOfBytes = in.readInt();
-    if (numOfBytes == 0) {
-      this.path = EMPTY_NAME;
-    } else {
-      this.path = new byte[numOfBytes];
-      in.readFully(path);
-    }
-    this.length = in.readLong();
-    this.isdir = in.readBoolean();
-    this.block_replication = in.readShort();
-    blocksize = in.readLong();
-    modification_time = in.readLong();
-    access_time = in.readLong();
-    permission.readFields(in);
-    owner = Text.readString(in);
-    group = Text.readString(in);
-    if (in.readBoolean()) {
-      numOfBytes = in.readInt();
-      this.symlink = new byte[numOfBytes];
-      in.readFully(symlink);
-    }
-  }
-
-}

+ 0 - 116
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/HdfsLocatedFileStatusWritable.java

@@ -1,116 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/** 
- * Interface that represents the over the wire information
- * including block locations for a file.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class HdfsLocatedFileStatusWritable extends HdfsFileStatusWritable {
-  private LocatedBlocksWritable locations;
-  
-  /**
-   * Default constructor
-   */
-  public HdfsLocatedFileStatusWritable() {
-  }
-  
-  /**
-   * 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 locations block locations
-   */
-  public HdfsLocatedFileStatusWritable(long length, boolean isdir,
-      int block_replication,
-	    long blocksize, long modification_time, long access_time,
-	    FsPermissionWritable permission, String owner, String group, 
-	    byte[] symlink, byte[] path, LocatedBlocksWritable locations) {
-	  super(length, isdir, block_replication, blocksize, modification_time,
-		  access_time, permission, owner, group, symlink, path);
-    this.locations = locations;
-	}
-  
-  static public org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus
-  convertLocatedHdfsFileStatus(HdfsLocatedFileStatusWritable fs) {
-  if (fs == null) return null;
-  return new org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus(fs.getLen(),
-      fs.isDir(), fs.getReplication(), fs.getBlockSize(),
-      fs.getModificationTime(), fs.getAccessTime(), 
-      FsPermissionWritable.convertPermission(fs.getPermission()),
-      fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
-      fs.getLocalNameInBytes(),
-      LocatedBlocksWritable.convertLocatedBlocks(fs.getBlockLocations()));
-}
-  
-  static public HdfsLocatedFileStatusWritable convertLocatedHdfsFileStatus(
-      org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus fs) {
-  if (fs == null) return null;
-  return new HdfsLocatedFileStatusWritable(fs.getLen(),
-      fs.isDir(), fs.getReplication(), fs.getBlockSize(),
-      fs.getModificationTime(), fs.getAccessTime(), 
-      org.apache.hadoop.hdfs.protocolR23Compatible.FsPermissionWritable.
-        convertPermission(fs.getPermission()),
-      fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
-      fs.getLocalNameInBytes(), 
-      LocatedBlocksWritable.convertLocatedBlocks(fs.getBlockLocations()));
-}
-	
-	public LocatedBlocksWritable getBlockLocations() {
-		return locations;
-	}
-	
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    if (!isDir() && !isSymlink()) {
-      locations.write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    if (!isDir() && !isSymlink()) {
-      locations = new LocatedBlocksWritable();
-      locations.readFields(in);
-    }
-  }
-}

+ 0 - 114
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolServerSideTranslatorR23.java

@@ -1,114 +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.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-
-/**
- * This class is used on the server side. Calls come across the wire for the
- * protocol family of Release 23 onwards. This class translates the R23 data
- * types to the native data types used inside the NN as specified in the generic
- * JournalProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class JournalProtocolServerSideTranslatorR23 implements
-    JournalWireProtocol {
-  final private JournalProtocol server;
-
-  /**
-   * Constructor
-   * 
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public JournalProtocolServerSideTranslatorR23(JournalProtocol server)
-      throws IOException {
-    this.server = server;
-  }
-
-  /**
-   * The client side will redirect getProtocolSignature to
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call getProtocolVersion
-   * and possibly in the future getProtocolSignature. Hence we still implement
-   * it even though the end client's call will never reach here.
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link JournalProtocol}
-     * 
-     */
-    if (!protocol.equals(RPC.getProtocolName(JournalWireProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(JournalWireProtocol.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        JournalWireProtocol.versionID, JournalWireProtocol.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable 
-          getProtocolSignature2(
-      String protocol, long clientVersion, int clientMethodsHash)
-      throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link ClientNamenodeProtocol}
-     * 
-     */
-
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    if (protocol.equals(RPC.getProtocolName(JournalWireProtocol.class))) {
-      return JournalWireProtocol.versionID;
-    }
-    throw new IOException("Namenode Serverside implements " +
-        RPC.getProtocolName(JournalWireProtocol.class) +
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public void journal(NamenodeRegistrationWritable registration,
-      long firstTxnId, int numTxns, byte[] records) throws IOException {
-    server.journal(registration.convert(), firstTxnId, numTxns, records);
-  }
-
-  @Override
-  public void startLogSegment(NamenodeRegistrationWritable registration,
-      long txid) throws IOException {
-    server.startLogSegment(registration.convert(), txid);
-  }
-}

+ 0 - 80
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalProtocolTranslatorR23.java

@@ -1,80 +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.protocolR23Compatible;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-
-/**
- * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
- * while translating from the parameter types used in ClientProtocol to those
- * used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class JournalProtocolTranslatorR23 implements
-    JournalProtocol, Closeable {
-  private final JournalWireProtocol rpcProxy;
-
-  public JournalProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
-      Configuration conf) throws IOException {
-    rpcProxy = RPC.getProxy(JournalWireProtocol.class,
-        JournalWireProtocol.versionID, nameNodeAddr, conf);
-  }
-
-  @Override
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public void journal(NamenodeRegistration registration, long firstTxnId,
-      int numTxns, byte[] records) throws IOException {
-    rpcProxy.journal(NamenodeRegistrationWritable.convert(registration),
-        firstTxnId, numTxns, records);
-  }
-
-  @Override
-  public void startLogSegment(NamenodeRegistration registration, long txid)
-      throws IOException {
-    rpcProxy.startLogSegment(NamenodeRegistrationWritable.convert(registration),
-        txid);
-  }
-}

+ 0 - 81
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/JournalWireProtocol.java

@@ -1,81 +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.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.KerberosInfo;
-
-/**
- * This class defines the actual protocol used to communicate with the
- * NN via RPC using writable types.
- * The parameters in the methods which are specified in the
- * package are separate from those used internally in the NN and DFSClient
- * and hence need to be converted using {@link JournalProtocolTranslatorR23}
- * and {@link JournalProtocolServerSideTranslatorR23}.
- *
- */
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
-    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
-@InterfaceAudience.Private
-public interface JournalWireProtocol extends VersionedProtocol {
-  
-  /**
-   * The  rules for changing this protocol are the same as that for
-   * {@link ClientNamenodeWireProtocol} - see that java file for details.
-   */
-  public static final long versionID = 1L;
-
-  /**
-   * Journal edit records.
-   * This message is sent by the active name-node to the backup node
-   * via {@code EditLogBackupOutputStream} in order to synchronize meta-data
-   * changes with the backup namespace image.
-   * 
-   * @param registration active node registration
-   * @param firstTxnId the first transaction of this batch
-   * @param numTxns number of transactions
-   * @param records byte array containing serialized journal records
-   */
-  public void journal(NamenodeRegistrationWritable registration,
-                      long firstTxnId,
-                      int numTxns,
-                      byte[] records) throws IOException;
-
-  /**
-   * Notify the BackupNode that the NameNode has rolled its edit logs
-   * and is now writing a new log segment.
-   * @param registration the registration of the active NameNode
-   * @param txid the first txid in the new log
-   */
-  public void startLogSegment(NamenodeRegistrationWritable registration,
-      long txid) throws IOException;
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
-           getProtocolSignature2(String protocol, long clientVersion,
-      int clientMethodsHash) throws IOException;
-}

+ 0 - 253
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlockWritable.java

@@ -1,253 +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.protocolR23Compatible;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.io.*;
-
-import java.io.*;
-import java.util.ArrayList;
-import java.util.List;
-
-/****************************************************
- * A LocatedBlock is a pair of Block, DatanodeInfo[]
- * objects.  It tells where to find a Block.
- * 
- ****************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class LocatedBlockWritable implements Writable {
-
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (LocatedBlockWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new LocatedBlockWritable(); }
-       });
-  }
-
-  private ExtendedBlockWritable b;
-  private long offset;  // offset of the first byte of the block in the file
-  private DatanodeInfoWritable[] locs;
-  // corrupt flag is true if all of the replicas of a block are corrupt.
-  // else false. If block has few corrupt replicas, they are filtered and 
-  // their locations are not part of this object
-  private boolean corrupt;
-  private TokenWritable blockToken = new TokenWritable();
-
-  
-  static public org.apache.hadoop.hdfs.protocol.LocatedBlock
-    convertLocatedBlock(LocatedBlockWritable lb) {
-    if (lb == null) return null;
-    org.apache.hadoop.hdfs.protocol.LocatedBlock result =  
-        new org.apache.hadoop.hdfs.protocol.LocatedBlock(ExtendedBlockWritable.
-            convertExtendedBlock(lb.getBlock()),
-        DatanodeInfoWritable.convertDatanodeInfo(
-            lb.getLocations()), lb.getStartOffset(), lb.isCorrupt());
-    
-    // Fill in the token
-    TokenWritable tok = lb.getBlockToken();
-    result.setBlockToken(
-        new org.apache.hadoop.security.token.Token<BlockTokenIdentifier>(
-            tok.getIdentifier(), tok.getPassword(), tok.getKind(),
-            tok.getService()));
-    return result;
-  }
-  
-  public static LocatedBlockWritable 
-    convertLocatedBlock(org.apache.hadoop.hdfs.protocol.LocatedBlock lb) {
-    if (lb == null) return null;
-    LocatedBlockWritable result =  
-        new LocatedBlockWritable(ExtendedBlockWritable.convertExtendedBlock(lb.getBlock()), 
-        DatanodeInfoWritable.convertDatanodeInfo(lb.getLocations()),
-        lb.getStartOffset(), lb.isCorrupt());
-    
-    // Fill in the token
-    org.apache.hadoop.security.token.Token<BlockTokenIdentifier> tok = 
-        lb.getBlockToken();
-    result.setBlockToken(new TokenWritable(tok.getIdentifier(), tok.getPassword(), 
-        tok.getKind(), tok.getService()));
-    return result;
-  }
-  
-  static public LocatedBlockWritable[] 
-      convertLocatedBlock(org.apache.hadoop.hdfs.protocol.LocatedBlock[] lb) {
-    if (lb == null) return null;
-    final int len = lb.length;
-    LocatedBlockWritable[] result = new LocatedBlockWritable[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = new LocatedBlockWritable(
-          ExtendedBlockWritable.convertExtendedBlock(lb[i].getBlock()),
-          DatanodeInfoWritable.convertDatanodeInfo(lb[i].getLocations()), 
-          lb[i].getStartOffset(), lb[i].isCorrupt());
-    }
-    return result;
-  }
-  
-  static public org.apache.hadoop.hdfs.protocol.LocatedBlock[] 
-      convertLocatedBlock(LocatedBlockWritable[] lb) {
-    if (lb == null) return null;
-    final int len = lb.length;
-    org.apache.hadoop.hdfs.protocol.LocatedBlock[] result = 
-        new org.apache.hadoop.hdfs.protocol.LocatedBlock[len];
-    for (int i = 0; i < len; ++i) {
-      result[i] = new org.apache.hadoop.hdfs.protocol.LocatedBlock(
-          ExtendedBlockWritable.convertExtendedBlock(lb[i].getBlock()),
-          DatanodeInfoWritable.convertDatanodeInfo(lb[i].getLocations()), 
-          lb[i].getStartOffset(), lb[i].isCorrupt());
-    }
-    return result;
-  }
-  
-  static public List<org.apache.hadoop.hdfs.protocol.LocatedBlock> 
-    convertLocatedBlock(
-        List<org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable> lb) {
-    if (lb == null) return null;
-    final int len = lb.size();
-    List<org.apache.hadoop.hdfs.protocol.LocatedBlock> result = 
-        new ArrayList<org.apache.hadoop.hdfs.protocol.LocatedBlock>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(LocatedBlockWritable.convertLocatedBlock(lb.get(i)));
-    }
-    return result;
-  }
-  
-  static public List<LocatedBlockWritable> 
-  convertLocatedBlock2(List<org.apache.hadoop.hdfs.protocol.LocatedBlock> lb) {
-    if (lb == null) return null;
-    final int len = lb.size();
-    List<LocatedBlockWritable> result = new ArrayList<LocatedBlockWritable>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(LocatedBlockWritable.convertLocatedBlock(lb.get(i)));
-    }
-    return result;
-  }
-  
-  public LocatedBlockWritable() {
-    this(new ExtendedBlockWritable(), new DatanodeInfoWritable[0], 0L, false);
-  }
-
-  public LocatedBlockWritable(ExtendedBlockWritable eb) {
-    this(eb, new DatanodeInfoWritable[0], 0L, false);
-  }
-  
-  public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs) {
-    this(b, locs, -1, false); // startOffset is unknown
-  }
-
-  public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs, long startOffset) {
-    this(b, locs, startOffset, false);
-  }
-
-  public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs, long startOffset, 
-                      boolean corrupt) {
-    this.b = b;
-    this.offset = startOffset;
-    this.corrupt = corrupt;
-    if (locs==null) {
-      this.locs = new DatanodeInfoWritable[0];
-    } else {
-      this.locs = locs;
-    }
-  }
-
-  public TokenWritable getBlockToken() {
-    return blockToken;
-  }
-
-  public void setBlockToken(TokenWritable token) {
-    this.blockToken = token;
-  }
-
-  public ExtendedBlockWritable getBlock() {
-    return b;
-  }
-
-  public DatanodeInfoWritable[] getLocations() {
-    return locs;
-  }
-  
-  public long getStartOffset() {
-    return offset;
-  }
-  
-  public long getBlockSize() {
-    return b.getNumBytes();
-  }
-
-  void setStartOffset(long value) {
-    this.offset = value;
-  }
-
-  void setCorrupt(boolean corrupt) {
-    this.corrupt = corrupt;
-  }
-  
-  public boolean isCorrupt() {
-    return this.corrupt;
-  }
-
-  ///////////////////////////////////////////
-  // Writable
-  ///////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    blockToken.write(out);
-    out.writeBoolean(corrupt);
-    out.writeLong(offset);
-    b.write(out);
-    out.writeInt(locs.length);
-    for (int i = 0; i < locs.length; i++) {
-      locs[i].write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    blockToken.readFields(in);
-    this.corrupt = in.readBoolean();
-    offset = in.readLong();
-    this.b = new ExtendedBlockWritable();
-    b.readFields(in);
-    int count = in.readInt();
-    this.locs = new DatanodeInfoWritable[count];
-    for (int i = 0; i < locs.length; i++) {
-      locs[i] = new DatanodeInfoWritable();
-      locs[i].readFields(in);
-    }
-  }
-
-  /** Read LocatedBlock from in. */
-  public static LocatedBlockWritable read(DataInput in) throws IOException {
-    final LocatedBlockWritable lb = new LocatedBlockWritable();
-    lb.readFields(in);
-    return lb;
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "{" + b
-        + "; getBlockSize()=" + getBlockSize()
-        + "; corrupt=" + corrupt
-        + "; offset=" + offset
-        + "; locs=" + java.util.Arrays.asList(locs)
-        + "}";
-  }
-}

+ 0 - 197
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/LocatedBlocksWritable.java

@@ -1,197 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Collection of blocks with their locations and the file length.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class LocatedBlocksWritable implements Writable {
-  private long fileLength;
-  private List<LocatedBlockWritable> blocks; // array of blocks with prioritized locations
-  private boolean underConstruction;
-  private LocatedBlockWritable lastLocatedBlock = null;
-  private boolean isLastBlockComplete = false;
-
-  public static org.apache.hadoop.hdfs.protocol.LocatedBlocks convertLocatedBlocks(
-      LocatedBlocksWritable lb) {
-    if (lb == null) {
-      return null;
-    }
-    return new org.apache.hadoop.hdfs.protocol.LocatedBlocks(
-        lb.getFileLength(), lb.isUnderConstruction(),
-        LocatedBlockWritable.convertLocatedBlock(lb.getLocatedBlocks()),
-        LocatedBlockWritable.convertLocatedBlock(lb.getLastLocatedBlock()),
-        lb.isLastBlockComplete());
-  }
-  
-  public static LocatedBlocksWritable convertLocatedBlocks(
-      org.apache.hadoop.hdfs.protocol.LocatedBlocks lb) {
-    if (lb == null) {
-      return null;
-    }
-    return new LocatedBlocksWritable(lb.getFileLength(), lb.isUnderConstruction(),
-        LocatedBlockWritable.convertLocatedBlock2(lb.getLocatedBlocks()),
-        LocatedBlockWritable.convertLocatedBlock(lb.getLastLocatedBlock()),
-        lb.isLastBlockComplete());
-  }
-  
-  public LocatedBlocksWritable() {
-    this(0, false, null, null, false);
-  }
-  
-  /** public Constructor */
-  public LocatedBlocksWritable(long flength, boolean isUnderConstuction,
-      List<LocatedBlockWritable> blks, 
-      LocatedBlockWritable lastBlock, boolean isLastBlockCompleted) {
-    fileLength = flength;
-    blocks = blks;
-    underConstruction = isUnderConstuction;
-    this.lastLocatedBlock = lastBlock;
-    this.isLastBlockComplete = isLastBlockCompleted;
-  }
-  
-  /**
-   * Get located blocks.
-   */
-  public List<LocatedBlockWritable> getLocatedBlocks() {
-    return blocks;
-  }
-  
-  /** Get the last located block. */
-  public LocatedBlockWritable getLastLocatedBlock() {
-    return lastLocatedBlock;
-  }
-  
-  /** Is the last block completed? */
-  public boolean isLastBlockComplete() {
-    return isLastBlockComplete;
-  }
-
-  /**
-   * Get located block.
-   */
-  public LocatedBlockWritable get(int index) {
-    return blocks.get(index);
-  }
-  
-  /**
-   * Get number of located blocks.
-   */
-  public int locatedBlockCount() {
-    return blocks == null ? 0 : blocks.size();
-  }
-
-  /**
-   * Get file length
-   */
-  public long getFileLength() {
-    return this.fileLength;
-  }
-
-  /**
-   * Return ture if file was under construction when 
-   * this LocatedBlocks was constructed, false otherwise.
-   */
-  public boolean isUnderConstruction() {
-    return underConstruction;
-  }
-
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (LocatedBlocksWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new LocatedBlocksWritable(); }
-       });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(this.fileLength);
-    out.writeBoolean(underConstruction);
-
-    //write the last located block
-    final boolean isNull = lastLocatedBlock == null;
-    out.writeBoolean(isNull);
-    if (!isNull) {
-      lastLocatedBlock.write(out);
-    }
-    out.writeBoolean(isLastBlockComplete);
-
-    // write located blocks
-    int nrBlocks = locatedBlockCount();
-    out.writeInt(nrBlocks);
-    if (nrBlocks == 0) {
-      return;
-    }
-    for (LocatedBlockWritable blk : this.blocks) {
-      blk.write(out);
-    }
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.fileLength = in.readLong();
-    underConstruction = in.readBoolean();
-
-    //read the last located block
-    final boolean isNull = in.readBoolean();
-    if (!isNull) {
-      lastLocatedBlock = LocatedBlockWritable.read(in);
-    }
-    isLastBlockComplete = in.readBoolean();
-
-    // read located blocks
-    int nrBlocks = in.readInt();
-    this.blocks = new ArrayList<LocatedBlockWritable>(nrBlocks);
-    for (int idx = 0; idx < nrBlocks; idx++) {
-      LocatedBlockWritable blk = new LocatedBlockWritable();
-      blk.readFields(in);
-      this.blocks.add(blk);
-    }
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder b = new StringBuilder(getClass().getSimpleName());
-    b.append("{")
-     .append("\n  fileLength=").append(fileLength)
-     .append("\n  underConstruction=").append(underConstruction)
-     .append("\n  blocks=").append(blocks)
-     .append("\n  lastLocatedBlock=").append(lastLocatedBlock)
-     .append("\n  isLastBlockComplete=").append(isLastBlockComplete)
-     .append("}");
-    return b.toString();
-  }
-}

+ 0 - 70
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeCommandWritable.java

@@ -1,70 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Base class for name-node command.
- * Issued by the name-node to notify other name-nodes what should be done.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class NamenodeCommandWritable implements Writable {
-  private int action;
-  static {
-    WritableFactories.setFactory(NamenodeCommandWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {return new NamenodeCommandWritable();}
-        });
-  }
-
-  public NamenodeCommandWritable() {
-  }
-
-  public NamenodeCommandWritable(int action) {
-    this.action = action;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(this.action);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.action = in.readInt();
-  }
-
-  public static NamenodeCommandWritable convert(NamenodeCommand cmd) {
-    return new NamenodeCommandWritable(cmd.getAction());
-  }
-
-  public NamenodeCommand convert() {
-    return new NamenodeCommand(action);
-  }
-}

+ 0 - 163
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolServerSideTranslatorR23.java

@@ -1,163 +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.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-
-/**
- * This class is used on the server side.
- * Calls come across the wire for the protocol family of Release 23 onwards.
- * This class translates the R23 data types to the internal data types used
- * inside the DN as specified in the generic NamenodeProtocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class NamenodeProtocolServerSideTranslatorR23 implements
-  NamenodeWireProtocol {
-  
-  final private NamenodeProtocol server;
-
-  /**
-   * @param server - the NN server
-   * @throws IOException
-   */
-  public NamenodeProtocolServerSideTranslatorR23(
-      NamenodeProtocol server) throws IOException {
-    this.server = server;
-  }
-  
-  /**
-   * the client side will redirect getProtocolSignature to 
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call
-   * getProtocolVersion and possibly in the future getProtocolSignature.
-   * Hence we still implement it even though the end client's call will
-   * never reach here.
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and
-     * signature is that of  {@link NamenodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(
-        NamenodeWireProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " + 
-          NamenodeWireProtocol.class + 
-          ". The following requested protocol is unknown: " + protocol);
-    }
-    
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-        NamenodeWireProtocol.versionID, 
-        NamenodeWireProtocol.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and
-     * signature is that of  {@link ClientNamenodeProtocol}
-     */
-   return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    if (protocol.equals(RPC.getProtocolName(
-        NamenodeWireProtocol.class))) {
-      return NamenodeWireProtocol.versionID; 
-    }
-    throw new IOException("Datanode Serverside implements " + 
-        NamenodeWireProtocol.class + 
-        ". The following requested protocol is unknown: " + protocol);
-  }
-
-  @Override
-  public BlocksWithLocationsWritable getBlocks(DatanodeInfoWritable datanode,
-      long size) throws IOException {
-    BlocksWithLocations locs = server.getBlocks(
-        DatanodeInfoWritable.convertDatanodeInfo(datanode), size);
-    return BlocksWithLocationsWritable.convert(locs);
-  }
-
-  @Override
-  public ExportedBlockKeysWritable getBlockKeys() throws IOException {
-    return ExportedBlockKeysWritable.convert(server.getBlockKeys());
-  }
-
-  @Override
-  public long getTransactionID() throws IOException {
-    return server.getTransactionID();
-  }
-
-  @Override
-  @SuppressWarnings("deprecation")
-  public CheckpointSignatureWritable rollEditLog() throws IOException {
-    return CheckpointSignatureWritable.convert(server.rollEditLog());
-  }
-
-  @Override
-  public NamespaceInfoWritable versionRequest() throws IOException {
-    return NamespaceInfoWritable.convert(server.versionRequest());
-  }
-
-  @Override
-  public void errorReport(NamenodeRegistrationWritable registration,
-      int errorCode, String msg) throws IOException {
-    server.errorReport(registration.convert(), errorCode, msg);
-  }
-
-  @Override
-  public NamenodeRegistrationWritable register(
-      NamenodeRegistrationWritable registration) throws IOException {
-    return NamenodeRegistrationWritable.convert(server
-        .register(registration.convert()));
-  }
-
-  @Override
-  public NamenodeCommandWritable startCheckpoint(
-      NamenodeRegistrationWritable registration) throws IOException {
-    return NamenodeCommandWritable.convert(server.startCheckpoint(registration
-        .convert()));
-  }
-
-  @Override
-  public void endCheckpoint(NamenodeRegistrationWritable registration,
-      CheckpointSignatureWritable sig) throws IOException {
-    server.endCheckpoint(registration.convert(), sig.convert());
-  }
-
-  @Override
-  public RemoteEditLogManifestWritable getEditLogManifest(long sinceTxId)
-      throws IOException {
-    return RemoteEditLogManifestWritable.convert(server
-        .getEditLogManifest(sinceTxId));
-  }
-}

+ 0 - 179
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolTranslatorR23.java

@@ -1,179 +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.protocolR23Compatible;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.io.retry.RetryProxy;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * This class forwards NN's NamenodeProtocol calls as RPC calls to the NN server
- * while translating from the parameter types used in NamenodeProtocol to those
- * used in protocolR23Compatile.*.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class NamenodeProtocolTranslatorR23 implements
-    NamenodeProtocol, Closeable {
-  final private NamenodeWireProtocol rpcProxy;
-
-  private static NamenodeWireProtocol createNamenode(
-      InetSocketAddress nameNodeAddr, Configuration conf,
-      UserGroupInformation ugi) throws IOException {
-    return RPC.getProxy(NamenodeWireProtocol.class,
-        NamenodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
-        NetUtils.getSocketFactory(conf, NamenodeWireProtocol.class));
-  }
-
-  /** Create a {@link NameNode} proxy */
-  static NamenodeWireProtocol createNamenodeWithRetry(
-      NamenodeWireProtocol rpcNamenode) {
-    RetryPolicy createPolicy = RetryPolicies
-        .retryUpToMaximumCountWithFixedSleep(5,
-            HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
-
-    Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap = 
-        new HashMap<Class<? extends Exception>, RetryPolicy>();
-    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
-        createPolicy);
-
-    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = 
-        new HashMap<Class<? extends Exception>, RetryPolicy>();
-    exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
-        .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
-            remoteExceptionToPolicyMap));
-    RetryPolicy methodPolicy = RetryPolicies.retryByException(
-        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
-    Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
-
-    methodNameToPolicyMap.put("create", methodPolicy);
-
-    return (NamenodeWireProtocol) RetryProxy.create(
-        NamenodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
-  }
-
-  public NamenodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
-      Configuration conf, UserGroupInformation ugi) throws IOException {
-    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
-  }
-
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocolName,
-      long clientVersion, int clientMethodHash)
-      throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocolName, clientVersion, clientMethodHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion) throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
-      throws IOException {
-    return rpcProxy.getBlocks(
-        DatanodeInfoWritable.convertDatanodeInfo(datanode), size).convert();
-  }
-
-  @Override
-  public ExportedBlockKeys getBlockKeys() throws IOException {
-    return rpcProxy.getBlockKeys().convert();
-  }
-
-  @Override
-  public long getTransactionID() throws IOException {
-    return rpcProxy.getTransactionID();
-  }
-
-  @Override
-  public CheckpointSignature rollEditLog() throws IOException {
-    return rpcProxy.rollEditLog().convert();
-  }
-
-  @Override
-  public NamespaceInfo versionRequest() throws IOException {
-    return rpcProxy.versionRequest().convert();
-  }
-
-  @Override
-  public void errorReport(NamenodeRegistration registration, int errorCode,
-      String msg) throws IOException {
-    rpcProxy.errorReport(NamenodeRegistrationWritable.convert(registration),
-        errorCode, msg);
-  }
-
-  @Override
-  public NamenodeRegistration register(NamenodeRegistration registration)
-      throws IOException {
-    return rpcProxy
-        .register(NamenodeRegistrationWritable.convert(registration)).convert();
-  }
-
-  @Override
-  public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
-      throws IOException {
-    return rpcProxy.startCheckpoint(
-        NamenodeRegistrationWritable.convert(registration)).convert();
-  }
-
-  @Override
-  public void endCheckpoint(NamenodeRegistration registration,
-      CheckpointSignature sig) throws IOException {
-    rpcProxy.endCheckpoint(NamenodeRegistrationWritable.convert(registration),
-        CheckpointSignatureWritable.convert(sig));
-  }
-
-  @Override
-  public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
-      throws IOException {
-    return rpcProxy.getEditLogManifest(sinceTxId).convert();
-  }
-}

+ 0 - 98
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeRegistrationWritable.java

@@ -1,98 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-
-/**
- * Information sent by a subordinate name-node to the active name-node
- * during the registration process. 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class NamenodeRegistrationWritable implements Writable {
-  private String rpcAddress;    // RPC address of the node
-  private String httpAddress;   // HTTP address of the node
-  private NamenodeRole role;    // node role
-  private StorageInfoWritable storageInfo;
-
-  public NamenodeRegistrationWritable() { }
-
-  public NamenodeRegistrationWritable(String address,
-                              String httpAddress,
-                              NamenodeRole role,
-                              StorageInfo storageInfo) {
-    this.rpcAddress = address;
-    this.httpAddress = httpAddress;
-    this.role = role;
-    this.storageInfo = StorageInfoWritable.convert(storageInfo);
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {
-    WritableFactories.setFactory
-      (NamenodeRegistrationWritable.class,
-       new WritableFactory() {
-          public Writable newInstance() {
-            return new NamenodeRegistrationWritable();
-          }
-       });
-  }
-
-  @Override // Writable
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, rpcAddress);
-    Text.writeString(out, httpAddress);
-    Text.writeString(out, role.name());
-    storageInfo.write(out);
-  }
-
-  @Override // Writable
-  public void readFields(DataInput in) throws IOException {
-    rpcAddress = Text.readString(in);
-    httpAddress = Text.readString(in);
-    role = NamenodeRole.valueOf(Text.readString(in));
-    storageInfo = new StorageInfoWritable();
-    storageInfo.readFields(in);
-  }
-
-  public static NamenodeRegistrationWritable convert(NamenodeRegistration reg) {
-    return new NamenodeRegistrationWritable(reg.getAddress(),
-        reg.getHttpAddress(), reg.getRole(), reg);
-  }
-
-  public NamenodeRegistration convert() {
-    return new NamenodeRegistration(rpcAddress, httpAddress,
-        storageInfo.convert(), role);
-  }
-}

+ 0 - 166
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java

@@ -1,166 +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.protocolR23Compatible;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.KerberosInfo;
-
-/*****************************************************************************
- * Protocol that a secondary NameNode uses to communicate with the NameNode.
- * It's used to get part of the name node state
- *****************************************************************************/
-/** 
- * This class defines the actual protocol used to communicate between namenodes.
- * The parameters in the methods which are specified in the
- * package are separate from those used internally in the DN and DFSClient
- * and hence need to be converted using {@link NamenodeProtocolTranslatorR23}
- * and {@link NamenodeProtocolServerSideTranslatorR23}.
- */
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
-    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
-@InterfaceAudience.Private
-public interface NamenodeWireProtocol extends VersionedProtocol {
-  /**
-   * The  rules for changing this protocol are the same as that for
-   * {@link ClientNamenodeWireProtocol} - see that java file for details.
-   */
-  public static final long versionID = 6L;
-
-  /**
-   * Get a list of blocks belonging to <code>datanode</code>
-   * whose total size equals <code>size</code>.
-   * 
-   * @see org.apache.hadoop.hdfs.server.balancer.Balancer
-   * @param datanode  a data node
-   * @param size      requested size
-   * @return          a list of blocks & their locations
-   * @throws RemoteException if size is less than or equal to 0 or
-   *                               datanode does not exist
-   */
-  public BlocksWithLocationsWritable getBlocks(DatanodeInfoWritable datanode,
-      long size) throws IOException;
-
-  /**
-   * Get the current block keys
-   * 
-   * @return ExportedBlockKeys containing current block keys
-   * @throws IOException 
-   */
-  public ExportedBlockKeysWritable getBlockKeys() throws IOException;
-
-  /**
-   * @return The most recent transaction ID that has been synced to
-   * persistent storage.
-   * @throws IOException
-   */
-  public long getTransactionID() throws IOException;
-
-  /**
-   * Closes the current edit log and opens a new one. The 
-   * call fails if the file system is in SafeMode.
-   * @throws IOException
-   * @return a unique token to identify this transaction.
-   */
-  public CheckpointSignatureWritable rollEditLog() throws IOException;
-
-  /**
-   * Request name-node version and storage information.
-   * @throws IOException
-   */
-  public NamespaceInfoWritable versionRequest() throws IOException;
-
-  /**
-   * Report to the active name-node an error occurred on a subordinate node.
-   * Depending on the error code the active node may decide to unregister the
-   * reporting node.
-   * 
-   * @param registration requesting node.
-   * @param errorCode indicates the error
-   * @param msg free text description of the error
-   * @throws IOException
-   */
-  public void errorReport(NamenodeRegistrationWritable registration,
-                          int errorCode, 
-                          String msg) throws IOException;
-
-  /** 
-   * Register a subordinate name-node like backup node.
-   *
-   * @return  {@link NamenodeRegistration} of the node,
-   *          which this node has just registered with.
-   */
-  public NamenodeRegistrationWritable register(
-      NamenodeRegistrationWritable registration) throws IOException;
-
-  /**
-   * A request to the active name-node to start a checkpoint.
-   * The name-node should decide whether to admit it or reject.
-   * The name-node also decides what should be done with the backup node
-   * image before and after the checkpoint.
-   * 
-   * @see CheckpointCommand
-   * @see NamenodeCommandWritable
-   * @see #ACT_SHUTDOWN
-   * 
-   * @param registration the requesting node
-   * @return {@link CheckpointCommand} if checkpoint is allowed.
-   * @throws IOException
-   */
-  public NamenodeCommandWritable startCheckpoint(
-      NamenodeRegistrationWritable registration) throws IOException;
-
-  /**
-   * A request to the active name-node to finalize
-   * previously started checkpoint.
-   * 
-   * @param registration the requesting node
-   * @param sig {@code CheckpointSignature} which identifies the checkpoint.
-   * @throws IOException
-   */
-  public void endCheckpoint(NamenodeRegistrationWritable registration,
-                            CheckpointSignatureWritable sig) throws IOException;
-  
-  
-  /**
-   * Return a structure containing details about all edit logs
-   * available to be fetched from the NameNode.
-   * @param sinceTxId return only logs that contain transactions >= sinceTxId
-   */
-  public RemoteEditLogManifestWritable getEditLogManifest(long sinceTxId)
-    throws IOException;
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
-           getProtocolSignature2(String protocol, 
-      long clientVersion,
-      int clientMethodsHash) throws IOException;
-}
-

+ 0 - 100
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java

@@ -1,100 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * NamespaceInfoWritable is returned by the name-node in reply 
- * to a data-node handshake.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class NamespaceInfoWritable extends StorageInfo {
-  private String  buildVersion;
-  private int distributedUpgradeVersion;
-  private String blockPoolID = "";
-  private StorageInfoWritable storageInfo;
-
-  public NamespaceInfoWritable() {
-    super();
-    buildVersion = null;
-  }
-  
-  public NamespaceInfoWritable(int nsID, String clusterID, String bpID, 
-      long cT, int duVersion) {
-    this.blockPoolID = bpID;
-    this.buildVersion = Storage.getBuildVersion();
-    this.distributedUpgradeVersion = duVersion;
-    storageInfo = new StorageInfoWritable(HdfsConstants.LAYOUT_VERSION, nsID,
-        clusterID, cT);
-  }
-  
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {  // register a ctor
-    WritableFactories.setFactory
-      (NamespaceInfoWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new NamespaceInfoWritable(); }
-       });
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    DeprecatedUTF8.writeString(out, buildVersion);
-    storageInfo.write(out);
-    out.writeInt(distributedUpgradeVersion);
-    WritableUtils.writeString(out, blockPoolID);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    buildVersion = DeprecatedUTF8.readString(in);
-    storageInfo.readFields(in);
-    distributedUpgradeVersion = in.readInt();
-    blockPoolID = WritableUtils.readString(in);
-  }
-
-  public static NamespaceInfoWritable convert(NamespaceInfo info) {
-    return new NamespaceInfoWritable(info.getNamespaceID(), info.getClusterID(),
-        info.getBlockPoolID(), info.getCTime(),
-        info.getDistributedUpgradeVersion());
-  }
-  
-  public NamespaceInfo convert() {
-    return new NamespaceInfo(namespaceID, clusterID, blockPoolID, cTime,
-        distributedUpgradeVersion);
-  }
-}

+ 0 - 90
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java

@@ -1,90 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- * An enumeration of logs available on a remote NameNode.
- */
-public class RemoteEditLogManifestWritable implements Writable {
-  private List<RemoteEditLogWritable> logs;
-  
-  static { // register a ctor
-    WritableFactories.setFactory(RemoteEditLogManifestWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new RemoteEditLogManifestWritable();
-          }
-        });
-  }
-  
-  public RemoteEditLogManifestWritable() {
-  }
-  
-  public RemoteEditLogManifestWritable(List<RemoteEditLogWritable> logs) {
-    this.logs = logs;
-  }
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(logs.size());
-    for (RemoteEditLogWritable log : logs) {
-      log.write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int numLogs = in.readInt();
-    logs = Lists.newArrayList();
-    for (int i = 0; i < numLogs; i++) {
-      RemoteEditLogWritable log = new RemoteEditLogWritable();
-      log.readFields(in);
-      logs.add(log);
-    }
-  }
-
-  public static RemoteEditLogManifestWritable convert(
-      RemoteEditLogManifest editLogManifest) {
-    List<RemoteEditLogWritable> list = Lists.newArrayList();
-    for (RemoteEditLog log : editLogManifest.getLogs()) {
-      list.add(RemoteEditLogWritable.convert(log));
-    }
-    return new RemoteEditLogManifestWritable(list);
-  }
-
-  public RemoteEditLogManifest convert() {
-    List<RemoteEditLog> list = Lists.newArrayList();
-    for (RemoteEditLogWritable log : logs) {
-      list.add(log.convert());
-    }
-    return new RemoteEditLogManifest(list);
-  }
-}

+ 0 - 69
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java

@@ -1,69 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-public class RemoteEditLogWritable implements Writable  {
-  private long startTxId;
-  private long endTxId;
-  
-  static { // register a ctor
-    WritableFactories.setFactory(RemoteEditLogWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new RemoteEditLogWritable();
-          }
-        });
-  }
-  
-  public RemoteEditLogWritable() {
-  }
-
-  public RemoteEditLogWritable(long startTxId, long endTxId) {
-    this.startTxId = startTxId;
-    this.endTxId = endTxId;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(startTxId);
-    out.writeLong(endTxId);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    startTxId = in.readLong();
-    endTxId = in.readLong();
-  }
-
-  public static RemoteEditLogWritable convert(RemoteEditLog log) {
-    return new RemoteEditLogWritable(log.getStartTxId(), log.getEndTxId());
-  }
-
-  public RemoteEditLog convert() {
-    return new RemoteEditLog(startTxId, endTxId);
-  }
-}

+ 0 - 86
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/StorageInfoWritable.java

@@ -1,86 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * Common writable class for storage information.
- */
-@InterfaceAudience.Private
-public class StorageInfoWritable implements Writable {
-  private int layoutVersion;
-  private int namespaceID;
-  private String clusterID;
-  private long cTime;
-  
-  public StorageInfoWritable () {
-    this(0, 0, "", 0L);
-  }
-  
-  public StorageInfoWritable(int layoutV, int nsID, String cid, long cT) {
-    layoutVersion = layoutV;
-    clusterID = cid;
-    namespaceID = nsID;
-    cTime = cT;
-  }
-  
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {
-    WritableFactories.setFactory(StorageInfoWritable.class,
-        new WritableFactory() {
-          public Writable newInstance() {
-            return new StorageInfoWritable();
-          }
-        });
-  }
-  
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(layoutVersion);
-    out.writeInt(namespaceID);
-    WritableUtils.writeString(out, clusterID);
-    out.writeLong(cTime);
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    layoutVersion = in.readInt();
-    namespaceID = in.readInt();
-    clusterID = WritableUtils.readString(in);
-    cTime = in.readLong();
-  }
-
-  public StorageInfo convert() {
-    return new StorageInfo(layoutVersion, namespaceID, clusterID, cTime);
-  }
-  
-  public static StorageInfoWritable convert(StorageInfo from) {
-    return new StorageInfoWritable(from.getLayoutVersion(),
-        from.getNamespaceID(), from.getClusterID(), from.getCTime());
-  }
-}

+ 0 - 208
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/TokenWritable.java

@@ -1,208 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.commons.codec.binary.Base64;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * The client-side form of the token.
- */
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Stable
-public class TokenWritable implements Writable {
-  private byte[] identifier;
-  private byte[] password;
-  private Text kind;
-  private Text service;
- 
-  /**
-   * Construct a token from the components.
-   * @param identifier the token identifier
-   * @param password the token's password
-   * @param kind the kind of token
-   * @param service the service for this token
-   */
-  public TokenWritable(byte[] identifier, byte[] password, Text kind, Text service) {
-    this.identifier = identifier;
-    this.password = password;
-    this.kind = kind;
-    this.service = service;
-  }
-
-  /**
-   * Default constructor
-   */
-  public TokenWritable() {
-    this(new byte[0], new byte[0], new Text(), new Text());
-  }
-
-  /**
-   * Get the token identifier
-   * @return the token identifier
-   */
-  public byte[] getIdentifier() {
-    return identifier;
-  }
-  
-  /**
-   * Get the token password/secret
-   * @return the token password/secret
-   */
-  public byte[] getPassword() {
-    return password;
-  }
-  
-  /**
-   * Get the token kind
-   * @return the kind of the token
-   */
-  public Text getKind() {
-    return kind;
-  }
-
-  /**
-   * Get the service on which the token is supposed to be used
-   * @return the service name
-   */
-  public Text getService() {
-    return service;
-  }
-  
-  /**
-   * Set the service on which the token is supposed to be used
-   * @param newService the service name
-   */
-  public void setService(Text newService) {
-    service = newService;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int len = WritableUtils.readVInt(in);
-    if (identifier == null || identifier.length != len) {
-      identifier = new byte[len];
-    }
-    in.readFully(identifier);
-    len = WritableUtils.readVInt(in);
-    if (password == null || password.length != len) {
-      password = new byte[len];
-    }
-    in.readFully(password);
-    kind.readFields(in);
-    service.readFields(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, identifier.length);
-    out.write(identifier);
-    WritableUtils.writeVInt(out, password.length);
-    out.write(password);
-    kind.write(out);
-    service.write(out);
-  }
-
-  /**
-   * Generate a string with the url-quoted base64 encoded serialized form
-   * of the Writable.
-   * @param obj the object to serialize
-   * @return the encoded string
-   * @throws IOException
-   */
-  private static String encodeWritable(Writable obj) throws IOException {
-    DataOutputBuffer buf = new DataOutputBuffer();
-    obj.write(buf);
-    Base64 encoder = new Base64(0, null, true);
-    byte[] raw = new byte[buf.getLength()];
-    System.arraycopy(buf.getData(), 0, raw, 0, buf.getLength());
-    return encoder.encodeToString(raw);
-  }
-  
-  /**
-   * Modify the writable to the value from the newValue
-   * @param obj the object to read into
-   * @param newValue the string with the url-safe base64 encoded bytes
-   * @throws IOException
-   */
-  private static void decodeWritable(Writable obj, 
-                                     String newValue) throws IOException {
-    Base64 decoder = new Base64(0, null, true);
-    DataInputBuffer buf = new DataInputBuffer();
-    byte[] decoded = decoder.decode(newValue);
-    buf.reset(decoded, decoded.length);
-    obj.readFields(buf);
-  }
-
-  /**
-   * Encode this token as a url safe string
-   * @return the encoded string
-   * @throws IOException
-   */
-  public String encodeToUrlString() throws IOException {
-    return encodeWritable(this);
-  }
-  
-  /**
-   * Decode the given url safe string into this token.
-   * @param newValue the encoded string
-   * @throws IOException
-   */
-  public void decodeFromUrlString(String newValue) throws IOException {
-    decodeWritable(this, newValue);
-  }
-  
-  private static void addBinaryBuffer(StringBuilder buffer, byte[] bytes) {
-    for (int idx = 0; idx < bytes.length; idx++) {
-      // if not the first, put a blank separator in
-      if (idx != 0) {
-        buffer.append(' ');
-      }
-      String num = Integer.toHexString(0xff & bytes[idx]);
-      // if it is only one digit, add a leading 0.
-      if (num.length() < 2) {
-        buffer.append('0');
-      }
-      buffer.append(num);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    buffer.append("Ident: ");
-    addBinaryBuffer(buffer, identifier);
-    buffer.append(", Kind: ");
-    buffer.append(kind.toString());
-    buffer.append(", Service: ");
-    buffer.append(service.toString());
-    return buffer.toString();
-  }
-}

+ 0 - 140
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/UpgradeStatusReportWritable.java

@@ -1,140 +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.protocolR23Compatible;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
-
-/**
- * Base upgrade upgradeStatus class.
- * 
- * Describes status of current upgrade.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class UpgradeStatusReportWritable implements Writable {
-  protected int version;
-  protected short upgradeStatus;
-  protected boolean finalized;
-  
-  public static UpgradeStatusReportWritable convert(
-      org.apache.hadoop.hdfs.server.common.UpgradeStatusReport r) {
-    if (r == null) return null;
-    return new UpgradeStatusReportWritable(
-        r.getVersion(), r.getUpgradeStatus(), r.isFinalized());
-  }
-  
-  public static org.apache.hadoop.hdfs.server.common.UpgradeStatusReport 
-    convert(UpgradeStatusReportWritable r) {
-    if (r == null) return null;
-    return new org.apache.hadoop.hdfs.server.common.UpgradeStatusReport(
-        r.getVersion(), r.getUpgradeStatus(), r.isFinalized());
-  }
-  
-  public UpgradeStatusReportWritable() {
-    this(0, (short)0, false);
-  }
-
-  public UpgradeStatusReportWritable(int version, short status, boolean isFinalized) {
-    this.version = version;
-    this.upgradeStatus = status;
-    this.finalized = isFinalized;
-  }
-
-  /**
-   * Get the layout version of the currently running upgrade.
-   * @return layout version
-   */
-  public int getVersion() {
-    return this.version;
-  }
-
-  /**
-   * Get upgrade upgradeStatus as a percentage of the total upgrade done.
-   */ 
-  public short getUpgradeStatus() {
-    return upgradeStatus;
-  }
-
-  /**
-   * Is current upgrade finalized.
-   * @return true if finalized or false otherwise.
-   */
-  public boolean isFinalized() {
-    return this.finalized;
-  }
-
-  /**
-   * Get upgradeStatus data as a text for reporting.
-   * Should be overloaded for a particular upgrade specific upgradeStatus data.
-   * 
-   * @param details true if upgradeStatus details need to be included, 
-   *                false otherwise
-   * @return text
-   */
-  public String getStatusText(boolean details) {
-    return "Upgrade for version " + getVersion() 
-            + (upgradeStatus<100 ? 
-              " is in progress. Status = " + upgradeStatus + "%" : 
-              " has been completed."
-              + "\nUpgrade is " + (finalized ? "" : "not ")
-              + "finalized.");
-  }
-
-  /**
-   * Print basic upgradeStatus details.
-   */
-  @Override
-  public String toString() {
-    return getStatusText(false);
-  }
-
-  /////////////////////////////////////////////////
-  // Writable
-  /////////////////////////////////////////////////
-  static {                                      // register a ctor
-    WritableFactories.setFactory
-      (UpgradeStatusReportWritable.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new UpgradeStatusReportWritable(); }
-       });
-  }
-
-  
-  // Note when upgrade has been finalized then the NN always
-  // returns a null as the report. 
-  // hence the isFinalized is serialized (ugly)
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(this.version);
-    out.writeShort(this.upgradeStatus);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.version = in.readInt();
-    this.upgradeStatus = in.readShort();
-  }
-}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
@@ -308,8 +309,7 @@ public class DelegationTokenSecretManager
     }
     }
 
 
     final InetSocketAddress addr = namenode.getNameNodeAddress();
     final InetSocketAddress addr = namenode.getNameNodeAddress();
-    final String s = addr.getAddress().getHostAddress() + ":" + addr.getPort();
-    token.setService(new Text(s));
+    SecurityUtil.setTokenService(token, addr);
     final Credentials c = new Credentials();
     final Credentials c = new Credentials();
     c.addToken(new Text(ugi.getShortUserName()), token);
     c.addToken(new Text(ugi.getShortUserName()), token);
     return c;
     return c;

+ 35 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -17,27 +17,38 @@
  */
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 
 /**
 /**
- * Internal class for block metadata.
+ * BlockInfo class maintains for a given block
+ * the {@link INodeFile} it is part of and datanodes where the replicas of 
+ * the block are stored.
  */
  */
-public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
+@InterfaceAudience.Private
+public class BlockInfo extends Block implements
+    LightWeightGSet.LinkedElement {
   private INodeFile inode;
   private INodeFile inode;
 
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
   private LightWeightGSet.LinkedElement nextLinkedElement;
   private LightWeightGSet.LinkedElement nextLinkedElement;
 
 
   /**
   /**
-   * This array contains triplets of references.
-   * For each i-th datanode the block belongs to
-   * triplets[3*i] is the reference to the DatanodeDescriptor
-   * and triplets[3*i+1] and triplets[3*i+2] are references 
-   * to the previous and the next blocks, respectively, in the 
-   * list of blocks belonging to this data-node.
+   * This array contains triplets of references. For each i-th datanode the
+   * block belongs to triplets[3*i] is the reference to the DatanodeDescriptor
+   * and triplets[3*i+1] and triplets[3*i+2] are references to the previous and
+   * the next blocks, respectively, in the list of blocks belonging to this
+   * data-node.
+   * 
+   * Using previous and next in Object triplets is done instead of a
+   * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
+   * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
+   * bytes using the triplets.
    */
    */
   private Object[] triplets;
   private Object[] triplets;
 
 
@@ -84,7 +95,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     return node;
     return node;
   }
   }
 
 
-  BlockInfo getPrevious(int index) {
+  private BlockInfo getPrevious(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+1];
     BlockInfo info = (BlockInfo)triplets[index*3+1];
@@ -104,22 +115,14 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     return info;
     return info;
   }
   }
 
 
-  void setDatanode(int index, DatanodeDescriptor node) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-    triplets[index*3] = node;
-  }
-
-  void setPrevious(int index, BlockInfo to) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
-    triplets[index*3+1] = to;
-  }
-
-  void setNext(int index, BlockInfo to) {
+  private void setDatanode(int index, DatanodeDescriptor node, BlockInfo previous,
+      BlockInfo next) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
-    triplets[index*3+2] = to;
+    int i = index * 3;
+    assert index >= 0 && i+2 < triplets.length : "Index is out of bound";
+    triplets[i] = node;
+    triplets[i+1] = previous;
+    triplets[i+2] = next;
   }
   }
 
 
   /**
   /**
@@ -130,7 +133,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * @param to - block to be set to previous on the list of blocks
    * @param to - block to be set to previous on the list of blocks
    * @return current previous block on the list of blocks
    * @return current previous block on the list of blocks
    */
    */
-  BlockInfo getSetPrevious(int index, BlockInfo to) {
+  private BlockInfo setPrevious(int index, BlockInfo to) {
 	assert this.triplets != null : "BlockInfo is not initialized";
 	assert this.triplets != null : "BlockInfo is not initialized";
 	assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
 	assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+1];
     BlockInfo info = (BlockInfo)triplets[index*3+1];
@@ -146,7 +149,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * @param to - block to be set to next on the list of blocks
    * @param to - block to be set to next on the list of blocks
    *    * @return current next block on the list of blocks
    *    * @return current next block on the list of blocks
    */
    */
-  BlockInfo getSetNext(int index, BlockInfo to) {
+  private BlockInfo setNext(int index, BlockInfo to) {
 	assert this.triplets != null : "BlockInfo is not initialized";
 	assert this.triplets != null : "BlockInfo is not initialized";
 	assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
 	assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+2];
     BlockInfo info = (BlockInfo)triplets[index*3+2];
@@ -198,9 +201,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
       return false;
       return false;
     // find the last null node
     // find the last null node
     int lastNode = ensureCapacity(1);
     int lastNode = ensureCapacity(1);
-    setDatanode(lastNode, node);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
+    setDatanode(lastNode, node, null, null);
     return true;
     return true;
   }
   }
 
 
@@ -216,13 +217,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     // find the last not null node
     // find the last not null node
     int lastNode = numNodes()-1; 
     int lastNode = numNodes()-1; 
     // replace current node triplet by the lastNode one 
     // replace current node triplet by the lastNode one 
-    setDatanode(dnIndex, getDatanode(lastNode));
-    setNext(dnIndex, getNext(lastNode)); 
-    setPrevious(dnIndex, getPrevious(lastNode)); 
+    setDatanode(dnIndex, getDatanode(lastNode), getPrevious(lastNode),
+        getNext(lastNode));
     // set the last triplet to null
     // set the last triplet to null
-    setDatanode(lastNode, null);
-    setNext(lastNode, null); 
-    setPrevious(lastNode, null); 
+    setDatanode(lastNode, null, null, null);
     return true;
     return true;
   }
   }
 
 
@@ -300,8 +298,8 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     if (head == this) {
     if (head == this) {
       return this;
       return this;
     }
     }
-    BlockInfo next = this.getSetNext(curIndex, head);
-    BlockInfo prev = this.getSetPrevious(curIndex, null);
+    BlockInfo next = this.setNext(curIndex, head);
+    BlockInfo prev = this.setPrevious(curIndex, null);
 
 
     head.setPrevious(headIndex, this);
     head.setPrevious(headIndex, this);
     prev.setNext(prev.findDatanode(dn), next);
     prev.setNext(prev.findDatanode(dn), next);
@@ -331,7 +329,6 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
 
 
   /**
   /**
    * Convert a complete block to an under construction block.
    * Convert a complete block to an under construction block.
-   * 
    * @return BlockInfoUnderConstruction -  an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
    */
   public BlockInfoUnderConstruction convertToBlockUnderConstruction(
   public BlockInfoUnderConstruction convertToBlockUnderConstruction(

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -157,9 +157,6 @@ public class BlockInfoUnderConstruction extends BlockInfo {
   BlockInfo convertToCompleteBlock() throws IOException {
   BlockInfo convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != BlockUCState.COMPLETE :
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "Trying to convert a COMPLETE block";
       "Trying to convert a COMPLETE block";
-    if(getBlockUCState() != BlockUCState.COMMITTED)
-      throw new IOException(
-          "Cannot complete block: block has not been COMMITTED by the client");
     return new BlockInfo(this);
     return new BlockInfo(this);
   }
   }
 
 

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -497,6 +497,9 @@ public class BlockManager {
     if (!force && numNodes < minReplication)
     if (!force && numNodes < minReplication)
       throw new IOException("Cannot complete block: " +
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
           "block does not satisfy minimal replication requirement.");
+    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
+      throw new IOException(
+          "Cannot complete block: block has not been COMMITTED by the client");
     BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     // replace penultimate block in file
     // replace penultimate block in file
     fileINode.setBlock(blkIndex, completeBlock);
     fileINode.setBlock(blkIndex, completeBlock);

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.authentication.util.KerberosName;
@@ -492,19 +493,18 @@ public class JspHelper {
     return UserGroupInformation.createRemoteUser(strings[0]);
     return UserGroupInformation.createRemoteUser(strings[0]);
   }
   }
 
 
-  private static String getNNServiceAddress(ServletContext context,
+  private static InetSocketAddress getNNServiceAddress(ServletContext context,
       HttpServletRequest request) {
       HttpServletRequest request) {
     String namenodeAddressInUrl = request.getParameter(NAMENODE_ADDRESS);
     String namenodeAddressInUrl = request.getParameter(NAMENODE_ADDRESS);
     InetSocketAddress namenodeAddress = null;
     InetSocketAddress namenodeAddress = null;
     if (namenodeAddressInUrl != null) {
     if (namenodeAddressInUrl != null) {
-      namenodeAddress = DFSUtil.getSocketAddress(namenodeAddressInUrl);
+      namenodeAddress = NetUtils.createSocketAddr(namenodeAddressInUrl);
     } else if (context != null) {
     } else if (context != null) {
       namenodeAddress = NameNodeHttpServer.getNameNodeAddressFromContext(
       namenodeAddress = NameNodeHttpServer.getNameNodeAddressFromContext(
           context); 
           context); 
     }
     }
     if (namenodeAddress != null) {
     if (namenodeAddress != null) {
-      return (namenodeAddress.getAddress().getHostAddress() + ":" 
-          + namenodeAddress.getPort());
+      return namenodeAddress;
     }
     }
     return null;
     return null;
   }
   }
@@ -547,9 +547,9 @@ public class JspHelper {
         Token<DelegationTokenIdentifier> token = 
         Token<DelegationTokenIdentifier> token = 
           new Token<DelegationTokenIdentifier>();
           new Token<DelegationTokenIdentifier>();
         token.decodeFromUrlString(tokenString);
         token.decodeFromUrlString(tokenString);
-        String serviceAddress = getNNServiceAddress(context, request);
+        InetSocketAddress serviceAddress = getNNServiceAddress(context, request);
         if (serviceAddress != null) {
         if (serviceAddress != null) {
-          token.setService(new Text(serviceAddress));
+          SecurityUtil.setTokenService(token, serviceAddress);
           token.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
           token.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
         }
         }
         ByteArrayInputStream buf = new ByteArrayInputStream(token
         ByteArrayInputStream buf = new ByteArrayInputStream(token

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -599,8 +599,12 @@ public abstract class Storage extends StorageInfo {
      * @throws IOException if locking fails.
      * @throws IOException if locking fails.
      */
      */
     FileLock tryLock() throws IOException {
     FileLock tryLock() throws IOException {
+      boolean deletionHookAdded = false;
       File lockF = new File(root, STORAGE_FILE_LOCK);
       File lockF = new File(root, STORAGE_FILE_LOCK);
-      lockF.deleteOnExit();
+      if (!lockF.exists()) {
+        lockF.deleteOnExit();
+        deletionHookAdded = true;
+      }
       RandomAccessFile file = new RandomAccessFile(lockF, "rws");
       RandomAccessFile file = new RandomAccessFile(lockF, "rws");
       FileLock res = null;
       FileLock res = null;
       try {
       try {
@@ -613,6 +617,12 @@ public abstract class Storage extends StorageInfo {
         file.close();
         file.close();
         throw e;
         throw e;
       }
       }
+      if (res != null && !deletionHookAdded) {
+        // If the file existed prior to our startup, we didn't
+        // call deleteOnExit above. But since we successfully locked
+        // the dir, we can take care of cleaning it up.
+        lockF.deleteOnExit();
+      }
       return res;
       return res;
     }
     }
 
 

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -42,6 +42,8 @@ import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
@@ -49,6 +51,13 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.StringUtils;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;

+ 19 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -41,10 +41,14 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
 import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -251,8 +255,10 @@ class BPServiceActor implements Runnable {
       }
       }
     }
     }
     if (receivedAndDeletedBlockArray != null) {
     if (receivedAndDeletedBlockArray != null) {
+      StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
+          bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
       bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
       bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
-          receivedAndDeletedBlockArray);
+          report);
       synchronized (receivedAndDeletedBlockList) {
       synchronized (receivedAndDeletedBlockList) {
         for (int i = 0; i < receivedAndDeletedBlockArray.length; i++) {
         for (int i = 0; i < receivedAndDeletedBlockArray.length; i++) {
           receivedAndDeletedBlockList.remove(receivedAndDeletedBlockArray[i]);
           receivedAndDeletedBlockList.remove(receivedAndDeletedBlockArray[i]);
@@ -354,8 +360,9 @@ class BPServiceActor implements Runnable {
 
 
       // Send block report
       // Send block report
       long brSendStartTime = now();
       long brSendStartTime = now();
-      cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), bReport
-          .getBlockListAsLongs());
+      StorageBlockReport[] report = { new StorageBlockReport(
+          bpRegistration.getStorageID(), bReport.getBlockListAsLongs()) };
+      cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), report);
 
 
       // Log the block report processing stats from Datanode perspective
       // Log the block report processing stats from Datanode perspective
       long brSendCost = now() - brSendStartTime;
       long brSendCost = now() - brSendStartTime;
@@ -388,13 +395,17 @@ class BPServiceActor implements Runnable {
   
   
   HeartbeatResponse sendHeartBeat() throws IOException {
   HeartbeatResponse sendHeartBeat() throws IOException {
     LOG.info("heartbeat: " + this);
     LOG.info("heartbeat: " + this);
-    return bpNamenode.sendHeartbeat(bpRegistration,
+    // reports number of failed volumes
+    StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
+        false,
         dn.getFSDataset().getCapacity(),
         dn.getFSDataset().getCapacity(),
         dn.getFSDataset().getDfsUsed(),
         dn.getFSDataset().getDfsUsed(),
         dn.getFSDataset().getRemaining(),
         dn.getFSDataset().getRemaining(),
-        dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId()),
+        dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) };
+    return bpNamenode.sendHeartbeat(bpRegistration, report,
         dn.getXmitsInProgress(),
         dn.getXmitsInProgress(),
-        dn.getXceiverCount(), dn.getFSDataset().getNumFailedVolumes());
+        dn.getXceiverCount(),
+        dn.getFSDataset().getNumFailedVolumes());
   }
   }
   
   
   //This must be called only by BPOfferService
   //This must be called only by BPOfferService
@@ -569,7 +580,8 @@ class BPServiceActor implements Runnable {
     while (shouldRun()) {
     while (shouldRun()) {
       try {
       try {
         // Use returned registration from namenode with updated machine name.
         // Use returned registration from namenode with updated machine name.
-        bpRegistration = bpNamenode.registerDatanode(bpRegistration);
+        bpRegistration = bpNamenode.registerDatanode(bpRegistration,
+            new DatanodeStorage[0]);
         break;
         break;
       } catch(SocketTimeoutException e) {  // namenode is busy
       } catch(SocketTimeoutException e) {  // namenode is busy
         LOG.info("Problem connecting to server: " + nnAddr);
         LOG.info("Problem connecting to server: " + nnAddr);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -42,7 +42,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class BlockMetadataHeader {
 public class BlockMetadataHeader {
 
 
-  static final short METADATA_VERSION = FSDataset.METADATA_VERSION;
+  public static final short VERSION = 1;
   
   
   /**
   /**
    * Header includes everything except the checksum(s) themselves.
    * Header includes everything except the checksum(s) themselves.
@@ -138,7 +138,7 @@ public class BlockMetadataHeader {
    */
    */
   static void writeHeader(DataOutputStream out, DataChecksum checksum)
   static void writeHeader(DataOutputStream out, DataChecksum checksum)
                          throws IOException {
                          throws IOException {
-    writeHeader(out, new BlockMetadataHeader(METADATA_VERSION, checksum));
+    writeHeader(out, new BlockMetadataHeader(VERSION, checksum));
   }
   }
 
 
   /**
   /**

+ 13 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -46,15 +46,14 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.FSVolumeInterface;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 
 
 /**
 /**
  * Performs two types of scanning:
  * Performs two types of scanning:
  * <li> Gets block files from the data directories and reconciles the
  * <li> Gets block files from the data directories and reconciles the
- * difference between the blocks on the disk and in memory in
- * {@link FSDataset}</li>
+ * difference between the blocks on the disk and in memory.</li>
  * <li> Scans the data directories for block files under a block pool
  * <li> Scans the data directories for block files under a block pool
  * and verifies that the files are not corrupt</li>
  * and verifies that the files are not corrupt</li>
  * This keeps track of blocks and their last verification times.
  * This keeps track of blocks and their last verification times.
@@ -78,7 +77,7 @@ class BlockPoolSliceScanner {
 
 
   private long scanPeriod = DEFAULT_SCAN_PERIOD_HOURS * 3600 * 1000;
   private long scanPeriod = DEFAULT_SCAN_PERIOD_HOURS * 3600 * 1000;
   private DataNode datanode;
   private DataNode datanode;
-  private FSDataset dataset;
+  private final FSDatasetInterface dataset;
   
   
   // sorted set
   // sorted set
   private TreeSet<BlockScanInfo> blockInfoSet;
   private TreeSet<BlockScanInfo> blockInfoSet;
@@ -137,8 +136,8 @@ class BlockPoolSliceScanner {
     }
     }
   }
   }
   
   
-  BlockPoolSliceScanner(DataNode datanode, FSDataset dataset, Configuration conf,
-      String bpid) {
+  BlockPoolSliceScanner(DataNode datanode, FSDatasetInterface dataset,
+      Configuration conf, String bpid) {
     this.datanode = datanode;
     this.datanode = datanode;
     this.dataset = dataset;
     this.dataset = dataset;
     this.blockPoolId  = bpid;
     this.blockPoolId  = bpid;
@@ -220,16 +219,16 @@ class BlockPoolSliceScanner {
      * otherwise, pick the first directory.
      * otherwise, pick the first directory.
      */
      */
     File dir = null;
     File dir = null;
-    List<FSVolume> volumes = dataset.volumes.getVolumes();
-    for (FSDataset.FSVolume vol : dataset.volumes.getVolumes()) {
-      File bpDir = vol.getBlockPoolSlice(blockPoolId).getDirectory();
+    List<FSVolumeInterface> volumes = dataset.getVolumes();
+    for (FSVolumeInterface vol : volumes) {
+      File bpDir = vol.getDirectory(blockPoolId);
       if (LogFileHandler.isFilePresent(bpDir, verificationLogFile)) {
       if (LogFileHandler.isFilePresent(bpDir, verificationLogFile)) {
         dir = bpDir;
         dir = bpDir;
         break;
         break;
       }
       }
     }
     }
     if (dir == null) {
     if (dir == null) {
-      dir = volumes.get(0).getBlockPoolSlice(blockPoolId).getDirectory();
+      dir = volumes.get(0).getDirectory(blockPoolId);
     }
     }
     
     
     try {
     try {
@@ -425,9 +424,8 @@ class BlockPoolSliceScanner {
         updateScanStatus(block.getLocalBlock(), ScanType.VERIFICATION_SCAN, false);
         updateScanStatus(block.getLocalBlock(), ScanType.VERIFICATION_SCAN, false);
 
 
         // If the block does not exists anymore, then its not an error
         // If the block does not exists anymore, then its not an error
-        if ( dataset.getFile(block.getBlockPoolId(), block.getLocalBlock()) == null ) {
-          LOG.info("Verification failed for " + block + ". Its ok since " +
-          "it not in datanode dataset anymore.");
+        if (!dataset.contains(block)) {
+          LOG.info(block + " is no longer in the dataset.");
           deleteBlock(block.getLocalBlock());
           deleteBlock(block.getLocalBlock());
           return;
           return;
         }
         }
@@ -578,8 +576,8 @@ class BlockPoolSliceScanner {
     bytesLeft += len;
     bytesLeft += len;
   }
   }
 
 
-  static File getCurrentFile(FSVolume vol, String bpid) throws IOException {
-    return LogFileHandler.getCurrentFile(vol.getBlockPoolSlice(bpid).getDirectory(),
+  static File getCurrentFile(FSVolumeInterface vol, String bpid) throws IOException {
+    return LogFileHandler.getCurrentFile(vol.getDirectory(bpid),
         BlockPoolSliceScanner.verificationLogFile);
         BlockPoolSliceScanner.verificationLogFile);
   }
   }
   
   

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -225,7 +225,7 @@ class BlockReceiver implements Closeable {
       cleanupBlock();
       cleanupBlock();
       
       
       // check if there is a disk error
       // check if there is a disk error
-      IOException cause = FSDataset.getCauseIfDiskError(ioe);
+      IOException cause = DatanodeUtil.getCauseIfDiskError(ioe);
       DataNode.LOG.warn("IOException in BlockReceiver constructor. Cause is ",
       DataNode.LOG.warn("IOException in BlockReceiver constructor. Cause is ",
           cause);
           cause);
       
       

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -226,7 +226,7 @@ class BlockSender implements java.io.Closeable {
         // read and handle the common header here. For now just a version
         // read and handle the common header here. For now just a version
         BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
         BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
         short version = header.getVersion();
         short version = header.getVersion();
-        if (version != FSDataset.METADATA_VERSION) {
+        if (version != BlockMetadataHeader.VERSION) {
           LOG.warn("Wrong version (" + version + ") for metadata file for "
           LOG.warn("Wrong version (" + version + ") for metadata file for "
               + block + " ignoring ...");
               + block + " ignoring ...");
         }
         }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockVolumeChoosingPolicy.java

@@ -22,7 +22,7 @@ import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.FSVolumeInterface;
 
 
 /**************************************************
 /**************************************************
  * BlockVolumeChoosingPolicy allows a DataNode to
  * BlockVolumeChoosingPolicy allows a DataNode to
@@ -46,7 +46,7 @@ public interface BlockVolumeChoosingPolicy {
    * @return the chosen volume to store the block.
    * @return the chosen volume to store the block.
    * @throws IOException when disks are unavailable or are full.
    * @throws IOException when disks are unavailable or are full.
    */
    */
-  public FSVolume chooseVolume(List<FSVolume> volumes, long blockSize)
+  public FSVolumeInterface chooseVolume(List<FSVolumeInterface> volumes, long blockSize)
     throws IOException;
     throws IOException;
 
 
 }
 }

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -27,12 +27,12 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponse;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 
 
 /**
 /**
  * DataBlockScanner manages block scanning for all the block pools. For each
  * DataBlockScanner manages block scanning for all the block pools. For each
@@ -44,7 +44,7 @@ import org.apache.commons.logging.LogFactory;
 public class DataBlockScanner implements Runnable {
 public class DataBlockScanner implements Runnable {
   public static final Log LOG = LogFactory.getLog(DataBlockScanner.class);
   public static final Log LOG = LogFactory.getLog(DataBlockScanner.class);
   private final DataNode datanode;
   private final DataNode datanode;
-  private final FSDataset dataset;
+  private final FSDatasetInterface dataset;
   private final Configuration conf;
   private final Configuration conf;
   
   
   /**
   /**
@@ -55,7 +55,7 @@ public class DataBlockScanner implements Runnable {
     new TreeMap<String, BlockPoolSliceScanner>();
     new TreeMap<String, BlockPoolSliceScanner>();
   Thread blockScannerThread = null;
   Thread blockScannerThread = null;
   
   
-  DataBlockScanner(DataNode datanode, FSDataset dataset, Configuration conf) {
+  DataBlockScanner(DataNode datanode, FSDatasetInterface dataset, Configuration conf) {
     this.datanode = datanode;
     this.datanode = datanode;
     this.dataset = dataset;
     this.dataset = dataset;
     this.conf = conf;
     this.conf = conf;
@@ -135,7 +135,7 @@ public class DataBlockScanner implements Runnable {
               .iterator();
               .iterator();
           while (bpidIterator.hasNext()) {
           while (bpidIterator.hasNext()) {
             String bpid = bpidIterator.next();
             String bpid = bpidIterator.next();
-            for (FSDataset.FSVolume vol : dataset.volumes.getVolumes()) {
+            for (FSDatasetInterface.FSVolumeInterface vol : dataset.getVolumes()) {
               try {
               try {
                 File currFile = BlockPoolSliceScanner.getCurrentFile(vol, bpid);
                 File currFile = BlockPoolSliceScanner.getCurrentFile(vol, bpid);
                 if (currFile.exists()) {
                 if (currFile.exists()) {

+ 9 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -126,7 +126,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.common.Util;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.VolumeInfo;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods;
@@ -444,11 +443,11 @@ public class DataNode extends Configured
     if (conf.getInt(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
     if (conf.getInt(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY,
                     DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT) < 0) {
                     DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT) < 0) {
       reason = "verification is turned off by configuration";
       reason = "verification is turned off by configuration";
-    } else if (!(data instanceof FSDataset)) {
-      reason = "verifcation is supported only with FSDataset";
+    } else if ("SimulatedFSDataset".equals(data.getClass().getSimpleName())) {
+      reason = "verifcation is not supported by SimulatedFSDataset";
     } 
     } 
     if (reason == null) {
     if (reason == null) {
-      blockScanner = new DataBlockScanner(this, (FSDataset)data, conf);
+      blockScanner = new DataBlockScanner(this, data, conf);
       blockScanner.start();
       blockScanner.start();
     } else {
     } else {
       LOG.info("Periodic Block Verification scan is disabled because " +
       LOG.info("Periodic Block Verification scan is disabled because " +
@@ -473,11 +472,11 @@ public class DataNode extends Configured
     if (conf.getInt(DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 
     if (conf.getInt(DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 
                     DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT) < 0) {
                     DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT) < 0) {
       reason = "verification is turned off by configuration";
       reason = "verification is turned off by configuration";
-    } else if (!(data instanceof FSDataset)) {
-      reason = "verification is supported only with FSDataset";
+    } else if ("SimulatedFSDataset".equals(data.getClass().getSimpleName())) {
+      reason = "verifcation is not supported by SimulatedFSDataset";
     } 
     } 
     if (reason == null) {
     if (reason == null) {
-      directoryScanner = new DirectoryScanner(this, (FSDataset) data, conf);
+      directoryScanner = new DirectoryScanner(this, data, conf);
       directoryScanner.start();
       directoryScanner.start();
     } else {
     } else {
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +
@@ -1203,7 +1202,8 @@ public class DataNode extends Configured
   }
   }
     
     
   /** Number of concurrent xceivers per node. */
   /** Number of concurrent xceivers per node. */
-  int getXceiverCount() {
+  @Override // DataNodeMXBean
+  public int getXceiverCount() {
     return threadGroup == null ? 0 : threadGroup.activeCount();
     return threadGroup == null ? 0 : threadGroup.activeCount();
   }
   }
   
   
@@ -2145,16 +2145,7 @@ public class DataNode extends Configured
    */
    */
   @Override // DataNodeMXBean
   @Override // DataNodeMXBean
   public String getVolumeInfo() {
   public String getVolumeInfo() {
-    final Map<String, Object> info = new HashMap<String, Object>();
-    Collection<VolumeInfo> volumes = ((FSDataset)this.data).getVolumeInfo();
-    for (VolumeInfo v : volumes) {
-      final Map<String, Object> innerInfo = new HashMap<String, Object>();
-      innerInfo.put("usedSpace", v.usedSpace);
-      innerInfo.put("freeSpace", v.freeSpace);
-      innerInfo.put("reservedSpace", v.reservedSpace);
-      info.put(v.directory, innerInfo);
-    }
-    return JSON.toString(info);
+    return JSON.toString(data.getVolumeInfoMap());
   }
   }
   
   
   @Override // DataNodeMXBean
   @Override // DataNodeMXBean

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java

@@ -70,5 +70,10 @@ public interface DataNodeMXBean {
    * @return the cluster id
    * @return the cluster id
    */
    */
   public String getClusterId();
   public String getClusterId();
-  
+
+  /**
+   * Returns an estimate of the number of Datanode threads
+   * actively transferring blocks.
+   */
+  public int getXceiverCount();
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -751,7 +751,7 @@ public class DataStorage extends Storage {
     Matcher matcher = PRE_GENSTAMP_META_FILE_PATTERN.matcher(oldFileName); 
     Matcher matcher = PRE_GENSTAMP_META_FILE_PATTERN.matcher(oldFileName); 
     if (matcher.matches()) {
     if (matcher.matches()) {
       //return the current metadata file name
       //return the current metadata file name
-      return FSDataset.getMetaFileName(matcher.group(1),
+      return DatanodeUtil.getMetaFileName(matcher.group(1),
           GenerationStamp.GRANDFATHER_GENERATION_STAMP); 
           GenerationStamp.GRANDFATHER_GENERATION_STAMP); 
     }
     }
     return oldFileName;
     return oldFileName;

Some files were not shown because too many files changed in this diff