You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2016/12/08 22:29:08 UTC

[01/50] [abbrv] hadoop git commit: YARN-5929. Missing scheduling policy in the FS queue metric. (Contributed by Yufei Gu via Daniel Templeton)

Repository: hadoop
Updated Branches:
  refs/heads/HADOOP-13345 013a3c454 -> 881de1fba


YARN-5929. Missing scheduling policy in the FS queue metric. (Contributed by Yufei Gu via Daniel Templeton)


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

Branch: refs/heads/HADOOP-13345
Commit: 5bd18c49bd5075fa20d24363dceea7828e3fa266
Parents: 2ff84a0
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Dec 2 13:35:09 2016 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Dec 2 13:55:42 2016 -0800

----------------------------------------------------------------------
 .../scheduler/fair/FSQueueMetrics.java          | 32 +++++++--
 .../scheduler/fair/TestFSQueueMetrics.java      | 69 ++++++++++++++++++++
 2 files changed, 97 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bd18c49/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java
index a970815..ca375f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueMetrics.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
@@ -169,6 +170,12 @@ public class FSQueueMetrics extends QueueMetrics {
     amResourceUsageVCores.set(resource.getVirtualCores());
   }
 
+  /**
+   * Get the scheduling policy.
+   *
+   * @return the scheduling policy
+   */
+  @Metric("Scheduling policy")
   public String getSchedulingPolicy() {
     return schedulingPolicy;
   }
@@ -181,21 +188,38 @@ public class FSQueueMetrics extends QueueMetrics {
   static FSQueueMetrics forQueue(String queueName, Queue parent,
       boolean enableUserMetrics, Configuration conf) {
     MetricsSystem ms = DefaultMetricsSystem.instance();
+    return forQueue(ms, queueName, parent, enableUserMetrics, conf);
+  }
+
+  /**
+   * Get the FS queue metric for the given queue. Create one and register it to
+   * metrics system if there isn't one for the queue.
+   *
+   * @param ms the metric system
+   * @param queueName queue name
+   * @param parent parent queue
+   * @param enableUserMetrics  if user metrics is needed
+   * @param conf configuration
+   * @return a FSQueueMetrics object
+   */
+  @VisibleForTesting
+  public synchronized
+  static FSQueueMetrics forQueue(MetricsSystem ms, String queueName,
+      Queue parent, boolean enableUserMetrics, Configuration conf) {
     QueueMetrics metrics = queueMetrics.get(queueName);
     if (metrics == null) {
       metrics = new FSQueueMetrics(ms, queueName, parent, enableUserMetrics, conf)
           .tag(QUEUE_INFO, queueName);
-      
+
       // Register with the MetricsSystems
       if (ms != null) {
         metrics = ms.register(
-                sourceName(queueName).toString(), 
-                "Metrics for queue: " + queueName, metrics);
+            sourceName(queueName).toString(),
+            "Metrics for queue: " + queueName, metrics);
       }
       queueMetrics.put(queueName, metrics);
     }
 
     return (FSQueueMetrics)metrics;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5bd18c49/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java
new file mode 100644
index 0000000..7ccfbc3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
+import org.apache.hadoop.metrics2.impl.MetricsRecords;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestQueueMetrics;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * The test class for {@link FSQueueMetrics}.
+ */
+public class TestFSQueueMetrics {
+  private static final Configuration CONF = new Configuration();
+
+  private MetricsSystem ms;
+
+  @Before public void setUp() {
+    ms = new MetricsSystemImpl();
+    QueueMetrics.clearQueueMetrics();
+  }
+
+  /**
+   * Test if the metric scheduling policy is set correctly.
+   */
+  @Test
+  public void testSchedulingPolicy() {
+    String queueName = "single";
+
+    FSQueueMetrics metrics = FSQueueMetrics.forQueue(ms, queueName, null, false,
+        CONF);
+    metrics.setSchedulingPolicy("drf");
+    checkSchedulingPolicy(queueName, "drf");
+
+    // test resetting the scheduling policy
+    metrics.setSchedulingPolicy("fair");
+    checkSchedulingPolicy(queueName, "fair");
+  }
+
+  private void checkSchedulingPolicy(String queueName, String policy) {
+    MetricsSource queueSource = TestQueueMetrics.queueSource(ms, queueName);
+    MetricsCollectorImpl collector = new MetricsCollectorImpl();
+    queueSource.getMetrics(collector, true);
+    MetricsRecords.assertTag(collector.getRecords().get(0), "SchedulingPolicy",
+        policy);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[40/50] [abbrv] hadoop git commit: HDFS-8630. WebHDFS : Support get/set/unset StoragePolicy. Contributed by Surendra Singh Lilhore.

Posted by st...@apache.org.
HDFS-8630. WebHDFS : Support get/set/unset StoragePolicy. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HADOOP-13345
Commit: ea2895f4ed5031809d856faa52e9de5b9501bdea
Parents: 72fe546
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Dec 7 15:52:16 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Dec 7 15:52:16 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  49 ++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  46 ++++
 .../hadoop/hdfs/web/resources/GetOpParam.java   |   3 +
 .../hadoop/hdfs/web/resources/PostOpParam.java  |   2 +
 .../hadoop/hdfs/web/resources/PutOpParam.java   |   1 +
 .../hdfs/web/resources/StoragePolicyParam.java  |  43 +++
 .../hadoop/fs/http/client/HttpFSFileSystem.java |  92 ++++++-
 .../hadoop/fs/http/server/FSOperations.java     | 130 +++++++++
 .../http/server/HttpFSParametersProvider.java   |  23 ++
 .../hadoop/fs/http/server/HttpFSServer.java     |  35 +++
 .../fs/http/client/BaseTestHttpFSWith.java      |  52 +++-
 .../web/resources/NamenodeWebHdfsMethods.java   |  39 ++-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |  27 ++
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    | 261 +++++++++++++++++++
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |  68 +++++
 .../hadoop/hdfs/web/resources/TestParam.java    |   8 +
 16 files changed, 871 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index a75f4f1..3690a86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -56,6 +57,8 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -588,4 +591,50 @@ class JsonUtilClient {
         lastLocatedBlock, isLastBlockComplete, null, null);
   }
 
+  public static Collection<BlockStoragePolicy> getStoragePolicies(
+      Map<?, ?> json) {
+    Map<?, ?> policiesJson = (Map<?, ?>) json.get("BlockStoragePolicies");
+    if (policiesJson != null) {
+      List<?> objs = (List<?>) policiesJson.get(BlockStoragePolicy.class
+          .getSimpleName());
+      if (objs != null) {
+        BlockStoragePolicy[] storagePolicies = new BlockStoragePolicy[objs
+            .size()];
+        for (int i = 0; i < objs.size(); i++) {
+          final Map<?, ?> m = (Map<?, ?>) objs.get(i);
+          BlockStoragePolicy blockStoragePolicy = toBlockStoragePolicy(m);
+          storagePolicies[i] = blockStoragePolicy;
+        }
+        return Arrays.asList(storagePolicies);
+      }
+    }
+    return new ArrayList<BlockStoragePolicy>(0);
+  }
+
+  public static BlockStoragePolicy toBlockStoragePolicy(Map<?, ?> m) {
+    byte id = ((Number) m.get("id")).byteValue();
+    String name = (String) m.get("name");
+    StorageType[] storageTypes = toStorageTypes((List<?>) m
+        .get("storageTypes"));
+    StorageType[] creationFallbacks = toStorageTypes((List<?>) m
+        .get("creationFallbacks"));
+    StorageType[] replicationFallbacks = toStorageTypes((List<?>) m
+        .get("replicationFallbacks"));
+    Boolean copyOnCreateFile = (Boolean) m.get("copyOnCreateFile");
+    return new BlockStoragePolicy(id, name, storageTypes, creationFallbacks,
+        replicationFallbacks, copyOnCreateFile.booleanValue());
+  }
+
+  private static StorageType[] toStorageTypes(List<?> list) {
+    if (list == null) {
+      return null;
+    } else {
+      StorageType[] storageTypes = new StorageType[list.size()];
+      for (int i = 0; i < list.size(); i++) {
+        storageTypes[i] = StorageType.parseStorageType((String) list.get(i));
+      }
+      return storageTypes;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 23804b7..fbb4bd6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -39,6 +39,7 @@ import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
@@ -82,6 +83,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -1715,6 +1717,50 @@ public class WebHdfsFileSystem extends FileSystem
         : tokenServiceName.toString();
   }
 
+  @Override
+  public void setStoragePolicy(Path p, String policyName) throws IOException {
+    if (policyName == null) {
+      throw new IOException("policyName == null");
+    }
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.SET_STORAGE_POLICY);
+    final HttpOpParam.Op op = PutOpParam.Op.SETSTORAGEPOLICY;
+    new FsPathRunner(op, p, new StoragePolicyParam(policyName)).run();
+  }
+
+  @Override
+  public Collection<BlockStoragePolicy> getAllStoragePolicies()
+      throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETALLSTORAGEPOLICY;
+    return new FsPathResponseRunner<Collection<BlockStoragePolicy>>(op, null) {
+      @Override
+      Collection<BlockStoragePolicy> decodeResponse(Map<?, ?> json)
+          throws IOException {
+        return JsonUtilClient.getStoragePolicies(json);
+      }
+    }.run();
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(Path src) throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETSTORAGEPOLICY;
+    return new FsPathResponseRunner<BlockStoragePolicy>(op, src) {
+      @Override
+      BlockStoragePolicy decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtilClient.toBlockStoragePolicy((Map<?, ?>) json
+            .get(BlockStoragePolicy.class.getSimpleName()));
+      }
+    }.run();
+  }
+
+  @Override
+  public void unsetStoragePolicy(Path src) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.UNSET_STORAGE_POLICY);
+    final HttpOpParam.Op op = PostOpParam.Op.UNSETSTORAGEPOLICY;
+    new FsPathRunner(op, src).run();
+  }
+
   @VisibleForTesting
   InetSocketAddress[] getResolvedNNAddr() {
     return nnAddrs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
index 635e6d7..9169ca8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
@@ -40,6 +40,9 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETTRASHROOT(false, HttpURLConnection.HTTP_OK),
     LISTXATTRS(false, HttpURLConnection.HTTP_OK),
 
+    GETALLSTORAGEPOLICY(false, HttpURLConnection.HTTP_OK),
+    GETSTORAGEPOLICY(false, HttpURLConnection.HTTP_OK),
+
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED),
 
     CHECKACCESS(false, HttpURLConnection.HTTP_OK),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
index 4719bf3..56a14c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
@@ -29,6 +29,8 @@ public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
 
     TRUNCATE(false, HttpURLConnection.HTTP_OK),
 
+    UNSETSTORAGEPOLICY(false, HttpURLConnection.HTTP_OK),
+
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
 
     final boolean doOutputAndRedirect;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
index f36f874..4bb48a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
@@ -50,6 +50,7 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
     DISALLOWSNAPSHOT(false, HttpURLConnection.HTTP_OK),
     CREATESNAPSHOT(false, HttpURLConnection.HTTP_OK),
     RENAMESNAPSHOT(false, HttpURLConnection.HTTP_OK),
+    SETSTORAGEPOLICY(false, HttpURLConnection.HTTP_OK),
 
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StoragePolicyParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StoragePolicyParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StoragePolicyParam.java
new file mode 100644
index 0000000..60555ee
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StoragePolicyParam.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/** policy parameter. */
+public class StoragePolicyParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "storagepolicy";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   *
+   * @param str
+   *          a string representation of the parameter value.
+   */
+  public StoragePolicyParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index 6cc3909..dad8df2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.fs.http.client;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 
@@ -34,12 +36,14 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PositionedReadable;
 import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.lib.wsrs.EnumSetParam;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -114,6 +118,7 @@ public class HttpFSFileSystem extends FileSystem
   public static final String XATTR_ENCODING_PARAM = "encoding";
   public static final String NEW_LENGTH_PARAM = "newlength";
   public static final String START_AFTER_PARAM = "startAfter";
+  public static final String POLICY_NAME_PARAM = "storagepolicy";
 
   public static final Short DEFAULT_PERMISSION = 0755;
   public static final String ACLSPEC_DEFAULT = "";
@@ -193,6 +198,9 @@ public class HttpFSFileSystem extends FileSystem
   public static final String PARTIAL_LISTING_JSON = "partialListing";
   public static final String REMAINING_ENTRIES_JSON = "remainingEntries";
 
+  public static final String STORAGE_POLICIES_JSON = "BlockStoragePolicies";
+  public static final String STORAGE_POLICY_JSON = "BlockStoragePolicy";
+
   public static final int HTTP_TEMPORARY_REDIRECT = 307;
 
   private static final String HTTP_GET = "GET";
@@ -212,7 +220,9 @@ public class HttpFSFileSystem extends FileSystem
     MODIFYACLENTRIES(HTTP_PUT), REMOVEACLENTRIES(HTTP_PUT),
     REMOVEDEFAULTACL(HTTP_PUT), REMOVEACL(HTTP_PUT), SETACL(HTTP_PUT),
     DELETE(HTTP_DELETE), SETXATTR(HTTP_PUT), GETXATTRS(HTTP_GET),
-    REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET), LISTSTATUS_BATCH(HTTP_GET);
+    REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET), LISTSTATUS_BATCH(HTTP_GET),
+    GETALLSTORAGEPOLICY(HTTP_GET), GETSTORAGEPOLICY(HTTP_GET),
+    SETSTORAGEPOLICY(HTTP_PUT), UNSETSTORAGEPOLICY(HTTP_POST);
 
     private String httpMethod;
 
@@ -1310,4 +1320,84 @@ public class HttpFSFileSystem extends FileSystem
         params, f, true);
     HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
+
+  @Override
+  public Collection<BlockStoragePolicy> getAllStoragePolicies()
+      throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.GETALLSTORAGEPOLICY.toString());
+    HttpURLConnection conn = getConnection(
+        Operation.GETALLSTORAGEPOLICY.getMethod(), params, new Path(getUri()
+            .toString(), "/"), false);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    return createStoragePolicies((JSONObject) json.get(STORAGE_POLICIES_JSON));
+  }
+
+  private Collection<BlockStoragePolicy> createStoragePolicies(JSONObject map)
+      throws IOException {
+    JSONArray jsonArray = (JSONArray) map.get(STORAGE_POLICY_JSON);
+    BlockStoragePolicy[] policies = new BlockStoragePolicy[jsonArray.size()];
+    for (int i = 0; i < jsonArray.size(); i++) {
+      policies[i] = createStoragePolicy((JSONObject) jsonArray.get(i));
+    }
+    return Arrays.asList(policies);
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(Path src) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.GETSTORAGEPOLICY.toString());
+    HttpURLConnection conn = getConnection(
+        Operation.GETSTORAGEPOLICY.getMethod(), params, src, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    return createStoragePolicy((JSONObject) json.get(STORAGE_POLICY_JSON));
+  }
+
+  private BlockStoragePolicy createStoragePolicy(JSONObject policyJson)
+      throws IOException {
+    byte id = ((Number) policyJson.get("id")).byteValue();
+    String name = (String) policyJson.get("name");
+    StorageType[] storageTypes = toStorageTypes((JSONArray) policyJson
+        .get("storageTypes"));
+    StorageType[] creationFallbacks = toStorageTypes((JSONArray) policyJson
+        .get("creationFallbacks"));
+    StorageType[] replicationFallbacks = toStorageTypes((JSONArray) policyJson
+        .get("replicationFallbacks"));
+    Boolean copyOnCreateFile = (Boolean) policyJson.get("copyOnCreateFile");
+    return new BlockStoragePolicy(id, name, storageTypes, creationFallbacks,
+        replicationFallbacks, copyOnCreateFile.booleanValue());
+  }
+
+  private StorageType[] toStorageTypes(JSONArray array) throws IOException {
+    if (array == null) {
+      return null;
+    } else {
+      List<StorageType> storageTypes = new ArrayList<StorageType>(array.size());
+      for (Object name : array) {
+        storageTypes.add(StorageType.parseStorageType((String) name));
+      }
+      return storageTypes.toArray(new StorageType[storageTypes.size()]);
+    }
+  }
+
+  @Override
+  public void setStoragePolicy(Path src, String policyName) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.SETSTORAGEPOLICY.toString());
+    params.put(POLICY_NAME_PARAM, policyName);
+    HttpURLConnection conn = getConnection(
+        Operation.SETSTORAGEPOLICY.getMethod(), params, src, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  @Override
+  public void unsetStoragePolicy(Path src) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.UNSETSTORAGEPOLICY.toString());
+    HttpURLConnection conn = getConnection(
+        Operation.UNSETSTORAGEPOLICY.getMethod(), params, src, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index ebdb73e..6de7012 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
@@ -26,12 +27,14 @@ import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.GlobFilter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.util.StringUtils;
@@ -42,6 +45,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -260,6 +264,46 @@ public class FSOperations {
     return json;
   }
 
+  @SuppressWarnings({ "unchecked" })
+  private static JSONObject storagePolicyToJSON(BlockStoragePolicySpi policy) {
+    BlockStoragePolicy p = (BlockStoragePolicy) policy;
+    JSONObject policyJson = new JSONObject();
+    policyJson.put("id", p.getId());
+    policyJson.put("name", p.getName());
+    policyJson.put("storageTypes", toJsonArray(p.getStorageTypes()));
+    policyJson.put("creationFallbacks", toJsonArray(p.getCreationFallbacks()));
+    policyJson.put("replicationFallbacks",
+        toJsonArray(p.getReplicationFallbacks()));
+    policyJson.put("copyOnCreateFile", p.isCopyOnCreateFile());
+    return policyJson;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static JSONArray toJsonArray(StorageType[] storageTypes) {
+    JSONArray jsonArray = new JSONArray();
+    for (StorageType type : storageTypes) {
+      jsonArray.add(type.toString());
+    }
+    return jsonArray;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static JSONObject storagePoliciesToJSON(
+      Collection<? extends BlockStoragePolicySpi> storagePolicies) {
+    JSONObject json = new JSONObject();
+    JSONArray jsonArray = new JSONArray();
+    JSONObject policies = new JSONObject();
+    if (storagePolicies != null) {
+      for (BlockStoragePolicySpi policy : storagePolicies) {
+        JSONObject policyMap = storagePolicyToJSON(policy);
+        jsonArray.add(policyMap);
+      }
+    }
+    policies.put(HttpFSFileSystem.STORAGE_POLICY_JSON, jsonArray);
+    json.put(HttpFSFileSystem.STORAGE_POLICIES_JSON, policies);
+    return json;
+  }
+
   /**
    * Executor that performs an append FileSystemAccess files system operation.
    */
@@ -1319,4 +1363,90 @@ public class FSOperations {
       return xAttrsToJSON(xattrs, encoding);
     }
   }
+
+  /**
+   * Executor that performs a getAllStoragePolicies FileSystemAccess files
+   * system operation.
+   */
+  @SuppressWarnings({ "unchecked" })
+  @InterfaceAudience.Private
+  public static class FSGetAllStoragePolicies implements
+      FileSystemAccess.FileSystemExecutor<JSONObject> {
+
+    @Override
+    public JSONObject execute(FileSystem fs) throws IOException {
+      Collection<? extends BlockStoragePolicySpi> storagePolicies = fs
+          .getAllStoragePolicies();
+      return storagePoliciesToJSON(storagePolicies);
+    }
+  }
+
+  /**
+   * Executor that performs a getStoragePolicy FileSystemAccess files system
+   * operation.
+   */
+  @SuppressWarnings({ "unchecked" })
+  @InterfaceAudience.Private
+  public static class FSGetStoragePolicy implements
+      FileSystemAccess.FileSystemExecutor<JSONObject> {
+
+    private Path path;
+
+    public FSGetStoragePolicy(String path) {
+      this.path = new Path(path);
+    }
+
+    @Override
+    public JSONObject execute(FileSystem fs) throws IOException {
+      BlockStoragePolicySpi storagePolicy = fs.getStoragePolicy(path);
+      JSONObject json = new JSONObject();
+      json.put(HttpFSFileSystem.STORAGE_POLICY_JSON,
+          storagePolicyToJSON(storagePolicy));
+      return json;
+    }
+  }
+
+  /**
+   * Executor that performs a setStoragePolicy FileSystemAccess files system
+   * operation.
+   */
+  @InterfaceAudience.Private
+  public static class FSSetStoragePolicy implements
+      FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+    private String policyName;
+
+    public FSSetStoragePolicy(String path, String policyName) {
+      this.path = new Path(path);
+      this.policyName = policyName;
+    }
+
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.setStoragePolicy(path, policyName);
+      return null;
+    }
+  }
+
+  /**
+   * Executor that performs a unsetStoragePolicy FileSystemAccess files system
+   * operation.
+   */
+  @InterfaceAudience.Private
+  public static class FSUnsetStoragePolicy implements
+      FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+
+    public FSUnsetStoragePolicy(String path) {
+      this.path = new Path(path);
+    }
+
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.unsetStoragePolicy(path);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
index 9e89405..a9d350a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
@@ -94,6 +94,11 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.LISTXATTRS, new Class[]{});
     PARAMS_DEF.put(Operation.LISTSTATUS_BATCH,
         new Class[]{StartAfterParam.class});
+    PARAMS_DEF.put(Operation.GETALLSTORAGEPOLICY, new Class[] {});
+    PARAMS_DEF.put(Operation.GETSTORAGEPOLICY, new Class[] {});
+    PARAMS_DEF.put(Operation.SETSTORAGEPOLICY,
+        new Class[] {PolicyNameParam.class});
+    PARAMS_DEF.put(Operation.UNSETSTORAGEPOLICY, new Class[] {});
   }
 
   public HttpFSParametersProvider() {
@@ -541,4 +546,22 @@ public class HttpFSParametersProvider extends ParametersProvider {
       super(NAME, null);
     }
   }
+
+  /**
+   * Class for policyName parameter.
+   */
+  @InterfaceAudience.Private
+  public static class PolicyNameParam extends StringParam {
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.POLICY_NAME_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public PolicyNameParam() {
+      super(NAME, null);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index 3d81ac2..f526053 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OperationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OverwriteParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OwnerParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.PermissionParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.PolicyNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.RecursiveParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ReplicationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SourcesParam;
@@ -346,6 +347,22 @@ public class HttpFSServer {
       response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
       break;
     }
+    case GETALLSTORAGEPOLICY: {
+      FSOperations.FSGetAllStoragePolicies command =
+          new FSOperations.FSGetAllStoragePolicies();
+      JSONObject json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case GETSTORAGEPOLICY: {
+      FSOperations.FSGetStoragePolicy command =
+          new FSOperations.FSGetStoragePolicy(path);
+      JSONObject json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
     default: {
       throw new IOException(
           MessageFormat.format("Invalid HTTP GET operation [{0}]", op.value()));
@@ -473,6 +490,14 @@ public class HttpFSServer {
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
       }
+      case UNSETSTORAGEPOLICY: {
+        FSOperations.FSUnsetStoragePolicy command =
+             new FSOperations.FSUnsetStoragePolicy(path);
+         fsExecute(user, command);
+         AUDIT_LOG.info("Unset storage policy [{}]", path);
+         response = Response.ok().build();
+         break;
+      }
       default: {
         throw new IOException(
           MessageFormat.format("Invalid HTTP POST operation [{0}]",
@@ -690,6 +715,16 @@ public class HttpFSServer {
         response = Response.ok().build();
         break;
       }
+      case SETSTORAGEPOLICY: {
+        String policyName = params.get(PolicyNameParam.NAME,
+            PolicyNameParam.class);
+        FSOperations.FSSetStoragePolicy command =
+            new FSOperations.FSSetStoragePolicy(path, policyName);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] to policy [{}]", path, policyName);
+        response = Response.ok().build();
+        break;
+      }
       default: {
         throw new IOException(
           MessageFormat.format("Invalid HTTP PUT operation [{0}]",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index e130c68..2d86794 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.http.client;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
@@ -35,6 +36,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
@@ -941,12 +944,56 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     assertFalse(httpStatus.isEncrypted());
   }
 
+  private void testStoragePolicy() throws Exception {
+    Assume.assumeFalse("Assume its not a local FS", isLocalFS());
+    FileSystem fs = FileSystem.get(getProxiedFSConf());
+    fs.mkdirs(getProxiedFSTestDir());
+    Path path = new Path(getProxiedFSTestDir(), "policy.txt");
+    FileSystem httpfs = getHttpFSFileSystem();
+    // test getAllStoragePolicies
+    BlockStoragePolicy[] dfsPolicies = (BlockStoragePolicy[]) fs
+       .getAllStoragePolicies().toArray();
+    BlockStoragePolicy[] httpPolicies = (BlockStoragePolicy[]) httpfs
+        .getAllStoragePolicies().toArray();
+    Assert.assertArrayEquals(
+        "Policy array returned from the DFS and HttpFS should be equals",
+        dfsPolicies, httpPolicies);
+
+    // test get/set/unset policies
+    DFSTestUtil.createFile(fs, path, 0, (short) 1, 0L);
+    // get defaultPolicy
+   BlockStoragePolicySpi defaultdfsPolicy = fs.getStoragePolicy(path);
+    // set policy through webhdfs
+    httpfs.setStoragePolicy(path, HdfsConstants.COLD_STORAGE_POLICY_NAME);
+    // get policy from dfs
+    BlockStoragePolicySpi dfsPolicy = fs.getStoragePolicy(path);
+    // get policy from webhdfs
+    BlockStoragePolicySpi httpFsPolicy = httpfs.getStoragePolicy(path);
+    Assert
+       .assertEquals(
+            "Storage policy returned from the get API should"
+            + " be same as set policy",
+            HdfsConstants.COLD_STORAGE_POLICY_NAME.toString(),
+            httpFsPolicy.getName());
+    Assert.assertEquals(
+        "Storage policy returned from the DFS and HttpFS should be equals",
+        httpFsPolicy, dfsPolicy);
+    // unset policy
+    httpfs.unsetStoragePolicy(path);
+    Assert
+       .assertEquals(
+            "After unset storage policy, the get API shoudld"
+            + " return the default policy",
+            defaultdfsPolicy, httpfs.getStoragePolicy(path));
+    fs.close();
+  }
+
   protected enum Operation {
     GET, OPEN, CREATE, APPEND, TRUNCATE, CONCAT, RENAME, DELETE, LIST_STATUS, 
     WORKING_DIRECTORY, MKDIRS, SET_TIMES, SET_PERMISSION, SET_OWNER, 
     SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY, FILEACLS, DIRACLS, SET_XATTR,
     GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION, LIST_STATUS_BATCH,
-    GETTRASHROOT
+    GETTRASHROOT, STORAGEPOLICY
   }
 
   private void operation(Operation op) throws Exception {
@@ -1029,6 +1076,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     case GETTRASHROOT:
       testTrashRoot();
       break;
+    case STORAGEPOLICY:
+      testStoragePolicy();
+      break;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 5d9b12a..e400847 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -414,14 +415,16 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(CreateFlagParam.NAME) @DefaultValue(CreateFlagParam.DEFAULT)
           final CreateFlagParam createFlagParam,
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
-          final NoRedirectParam noredirect
+          final NoRedirectParam noredirect,
+      @QueryParam(StoragePolicyParam.NAME) @DefaultValue(StoragePolicyParam
+          .DEFAULT) final StoragePolicyParam policyName
       ) throws IOException, InterruptedException {
     return put(ugi, delegation, username, doAsUser, ROOT, op, destination,
         owner, group, permission, unmaskedPermission, overwrite, bufferSize,
         replication, blockSize, modificationTime, accessTime, renameOptions,
         createParent, delegationTokenArgument, aclPermission, xattrName,
         xattrValue, xattrSetFlag, snapshotName, oldSnapshotName,
-        excludeDatanodes, createFlagParam, noredirect);
+        excludeDatanodes, createFlagParam, noredirect, policyName);
   }
 
   /** Validate all required params. */
@@ -499,7 +502,9 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(CreateFlagParam.NAME) @DefaultValue(CreateFlagParam.DEFAULT)
           final CreateFlagParam createFlagParam,
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
-          final NoRedirectParam noredirect
+          final NoRedirectParam noredirect,
+      @QueryParam(StoragePolicyParam.NAME) @DefaultValue(StoragePolicyParam
+          .DEFAULT) final StoragePolicyParam policyName
       ) throws IOException, InterruptedException {
 
     init(ugi, delegation, username, doAsUser, path, op, destination, owner,
@@ -507,7 +512,7 @@ public class NamenodeWebHdfsMethods {
         replication, blockSize, modificationTime, accessTime, renameOptions,
         delegationTokenArgument, aclPermission, xattrName, xattrValue,
         xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes,
-        createFlagParam, noredirect);
+        createFlagParam, noredirect, policyName);
 
     return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
@@ -519,7 +524,7 @@ public class NamenodeWebHdfsMethods {
               renameOptions, createParent, delegationTokenArgument,
               aclPermission, xattrName, xattrValue, xattrSetFlag,
               snapshotName, oldSnapshotName, excludeDatanodes,
-              createFlagParam, noredirect);
+              createFlagParam, noredirect, policyName);
       }
     });
   }
@@ -553,7 +558,8 @@ public class NamenodeWebHdfsMethods {
       final OldSnapshotNameParam oldSnapshotName,
       final ExcludeDatanodesParam exclDatanodes,
       final CreateFlagParam createFlagParam,
-      final NoRedirectParam noredirectParam
+      final NoRedirectParam noredirectParam,
+      final StoragePolicyParam policyName
       ) throws IOException, URISyntaxException {
 
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
@@ -706,6 +712,13 @@ public class NamenodeWebHdfsMethods {
       np.disallowSnapshot(fullpath);
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
+    case SETSTORAGEPOLICY: {
+      if (policyName.getValue() == null) {
+        throw new IllegalArgumentException("Storage policy name is empty.");
+      }
+      np.setStoragePolicy(fullpath, policyName.getValue());
+      return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
@@ -829,6 +842,10 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
+    case UNSETSTORAGEPOLICY: {
+      np.unsetStoragePolicy(fullpath);
+      return Response.ok().build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
@@ -1094,6 +1111,16 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(listing);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
+    case GETALLSTORAGEPOLICY: {
+      BlockStoragePolicy[] storagePolicies = np.getStoragePolicies();
+      final String js = JsonUtil.toJsonString(storagePolicies);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
+    case GETSTORAGEPOLICY: {
+      BlockStoragePolicy storagePolicy = np.getStoragePolicy(fullpath);
+      final String js = JsonUtil.toJsonString(storagePolicy);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index a0dadbd..05a5777 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -436,4 +436,31 @@ public class JsonUtil {
     return MAPPER.writeValueAsString(obj);
   }
 
+  public static String toJsonString(BlockStoragePolicy[] storagePolicies) {
+    final Map<String, Object> blockStoragePolicies = new TreeMap<>();
+    Object[] a = null;
+    if (storagePolicies != null && storagePolicies.length > 0) {
+      a = new Object[storagePolicies.length];
+      for (int i = 0; i < storagePolicies.length; i++) {
+        a[i] = toJsonMap(storagePolicies[i]);
+      }
+    }
+    blockStoragePolicies.put("BlockStoragePolicy", a);
+    return toJsonString("BlockStoragePolicies", blockStoragePolicies);
+  }
+
+  private static Object toJsonMap(BlockStoragePolicy blockStoragePolicy) {
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("id", blockStoragePolicy.getId());
+    m.put("name", blockStoragePolicy.getName());
+    m.put("storageTypes", blockStoragePolicy.getStorageTypes());
+    m.put("creationFallbacks", blockStoragePolicy.getCreationFallbacks());
+    m.put("replicationFallbacks", blockStoragePolicy.getReplicationFallbacks());
+    m.put("copyOnCreateFile", blockStoragePolicy.isCopyOnCreateFile());
+    return m;
+  }
+
+  public static String toJsonString(BlockStoragePolicy storagePolicy) {
+    return toJsonString(BlockStoragePolicy.class, toJsonMap(storagePolicy));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index eda1350..f91e89f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -54,6 +54,10 @@ WebHDFS REST API
         * [Set ACL](#Set_ACL)
         * [Get ACL Status](#Get_ACL_Status)
         * [Check access](#Check_access)
+        * [Get all Storage Policies](#Get_all_Storage_Policies)
+        * [Set Storage Policy](#Set_Storage_Policy)
+        * [Unset Storage Policy](#Unset_Storage_Policy)
+        * [Get Storage Policy](#Get_Storage_Policy)
     * [Extended Attributes(XAttrs) Operations](#Extended_AttributesXAttrs_Operations)
         * [Set XAttr](#Set_XAttr)
         * [Remove XAttr](#Remove_XAttr)
@@ -90,6 +94,9 @@ WebHDFS REST API
         * [RemoteException JSON Schema](#RemoteException_JSON_Schema)
         * [Token JSON Schema](#Token_JSON_Schema)
             * [Token Properties](#Token_Properties)
+        * [BlockStoragePolicy JSON Schema](#BlockStoragePolicy_JSON_Schema)
+            * [BlockStoragePolicy Properties](#BlockStoragePolicy_Properties)
+        * [BlockStoragePolicies JSON Schema](#BlockStoragePolicies_JSON_Schema)
     * [HTTP Query Parameter Dictionary](#HTTP_Query_Parameter_Dictionary)
         * [ACL Spec](#ACL_Spec)
         * [XAttr Name](#XAttr_Name)
@@ -124,6 +131,8 @@ WebHDFS REST API
         * [Token Service](#Token_Service)
         * [Username](#Username)
         * [NoRedirect](#NoRedirect)
+        * [Storage Policy](#Storage_Policy)
+        * [Start After](#Start_After)
 
 Document Conventions
 --------------------
@@ -156,6 +165,8 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop
     * [`GETXATTRS`](#Get_all_XAttrs) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttrs)
     * [`LISTXATTRS`](#List_all_XAttrs) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listXAttrs)
     * [`CHECKACCESS`](#Check_access) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).access)
+    * [`GETALLSTORAGEPOLICY`](#Get_all_Storage_Policies) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getAllStoragePolicies)
+    * [`GETSTORAGEPOLICY`](#Get_Storage_Policy) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStoragePolicy)
 *   HTTP PUT
     * [`CREATE`](#Create_and_Write_to_a_File) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).create)
     * [`MKDIRS`](#Make_a_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).mkdirs)
@@ -171,10 +182,12 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop
     * [`RENAMESNAPSHOT`](#Rename_Snapshot) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).renameSnapshot)
     * [`SETXATTR`](#Set_XAttr) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setXAttr)
     * [`REMOVEXATTR`](#Remove_XAttr) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).removeXAttr)
+    * [`SETSTORAGEPOLICY`](#Set_Storage_Policy) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setStoragePolicy)
 *   HTTP POST
     * [`APPEND`](#Append_to_a_File) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).append)
     * [`CONCAT`](#Concat_Files) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).concat)
     * [`TRUNCATE`](#Truncate_a_File) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).truncate)
+    * [`UNSETSTORAGEPOLICY`](#Unset_Storage_Policy) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).unsetStoragePolicy)
 *   HTTP DELETE
     * [`DELETE`](#Delete_a_FileDirectory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).delete)
     * [`DELETESNAPSHOT`](#Delete_Snapshot) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).deleteSnapshot)
@@ -1015,6 +1028,129 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getAclSta
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).access
 
+Storage Policy Operations
+-------------------------
+
+### Get all Storage Policies
+
+* Submit a HTTP GET request.
+
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1?op=GETALLSTORAGEPOLICY"
+
+    The client receives a response with a [`BlockStoragePolicies` JSON object](#BlockStoragePolicies_JSON_Schema):
+
+        HTTP/1.1 200 OK
+        Content-Type: application/json
+        Transfer-Encoding: chunked
+
+        {
+            "BlockStoragePolicies": {
+                "BlockStoragePolicy": [
+                   {
+                       "copyOnCreateFile": false,
+                       "creationFallbacks": [],
+                       "id": 2,
+                       "name": "COLD",
+                       "replicationFallbacks": [],
+                       "storageTypes": ["ARCHIVE"]
+                   },
+                   {
+                       "copyOnCreateFile": false,
+                       "creationFallbacks": ["DISK","ARCHIVE"],
+                       "id": 5,
+                       "name": "WARM",
+                       "replicationFallbacks": ["DISK","ARCHIVE"],
+                       "storageTypes": ["DISK","ARCHIVE"]
+                   },
+                   {
+                       "copyOnCreateFile": false,
+                       "creationFallbacks": [],
+                       "id": 7,
+                       "name": "HOT",
+                       "replicationFallbacks": ["ARCHIVE"],
+                       "storageTypes": ["DISK"]
+                   },
+                   {
+                       "copyOnCreateFile": false,
+                       "creationFallbacks": ["SSD","DISK"],
+                       "id": 10,"name": "ONE_SSD",
+                       "replicationFallbacks": ["SSD","DISK"],
+                       "storageTypes": ["SSD","DISK"]
+                   },
+                   {
+                       "copyOnCreateFile": false,
+                       "creationFallbacks": ["DISK"],
+                       "id": 12,
+                       "name": "ALL_SSD",
+                       "replicationFallbacks": ["DISK"],
+                       "storageTypes": ["SSD"]
+                   },
+                   {
+                       "copyOnCreateFile": true,
+                       "creationFallbacks": ["DISK"],
+                       "id": 15,
+                       "name": "LAZY_PERSIST",
+                       "replicationFallbacks": ["DISK"],
+                       "storageTypes": ["RAM_DISK","DISK"]
+                   }
+               ]
+           }
+        }
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getAllStoragePolicies
+
+### Set Storage Policy
+
+* Submit a HTTP PUT request.
+
+        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=SETSTORAGEPOLICY
+                                      &storagepolicy=<policy>"
+
+    The client receives a response with zero content length:
+
+        HTTP/1.1 200 OK
+        Content-Length: 0
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setStoragePolicy
+
+### Unset Storage Policy
+
+* Submit a HTTP POT request.
+
+        curl -i -X POST "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=UNSETSTORAGEPOLICY"
+
+    The client receives a response with zero content length:
+
+        HTTP/1.1 200 OK
+        Content-Length: 0
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).unsetStoragePolicy
+
+### Get Storage Policy
+
+* Submit a HTTP GET request.
+
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETSTORAGEPOLICY"
+
+    The client receives a response with a [`BlockStoragePolicy` JSON object](#BlockStoragePolicy_JSON_Schema):
+
+        HTTP/1.1 200 OK
+        Content-Type: application/json
+        Transfer-Encoding: chunked
+
+        {
+            "BlockStoragePolicy": {
+                "copyOnCreateFile": false,
+               "creationFallbacks": [],
+                "id":7,
+                "name":"HOT",
+                "replicationFallbacks":["ARCHIVE"],
+                "storageTypes":["DISK"]
+            }
+        }
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStoragePolicy
+
 Extended Attributes(XAttrs) Operations
 --------------------------------------
 
@@ -1871,6 +2007,107 @@ var tokenProperties =
 ```
 
 See also: [`Token` Properties](#Token_Properties), the note in [Delegation](#Delegation).
+### BlockStoragePolicy JSON Schema
+
+```json
+{
+  "name"      : "BlockStoragePolicy",
+  "properties":
+  {
+    "BlockStoragePolicy": blockStoragePolicyProperties      //See BlockStoragePolicy Properties
+  }
+}
+```
+
+See also: [`BlockStoragePolicy` Properties](#BlockStoragePolicy_Properties), [`GETSTORAGEPOLICY`](#Get_Storage_Policy)
+
+#### BlockStoragePolicy Properties
+
+JavaScript syntax is used to define `blockStoragePolicyProperties` so that it can be referred in both `BlockStoragePolicy` and `BlockStoragePolicies` JSON schemas.
+
+```javascript
+var blockStoragePolicyProperties =
+{
+  "type"      : "object",
+  "properties":
+  {
+    "id":
+    {
+      "description": "Policy ID.",
+      "type"       : "integer",
+      "required"   : true
+    },
+    "name":
+    {
+      "description": "Policy name.",
+      "type"       : "string",
+      "required"   : true
+    },
+    "storageTypes":
+    {
+      "description": "An array of storage types for block placement.",
+      "type"       : "array",
+      "required"   : true
+      "items"      :
+      {
+        "type": "string"
+      }
+    },
+    "replicationFallbacks":
+    {
+      "description": "An array of fallback storage types for replication.",
+      "type"       : "array",
+      "required"   : true
+      "items"      :
+      {
+        "type": "string"
+      }
+    },
+    "creationFallbacks":
+    {
+      "description": "An array of fallback storage types for file creation.",
+      "type"       : "array",
+      "required"   : true
+      "items"      :
+      {
+       "type": "string"
+      }
+    },
+    "copyOnCreateFile":
+    {
+      "description": "If set then the policy cannot be changed after file creation.",
+      "type"       : "boolean",
+      "required"   : true
+    }
+  }
+};
+```
+
+### BlockStoragePolicies JSON Schema
+
+A `BlockStoragePolicies` JSON object represents an array of `BlockStoragePolicy` JSON objects.
+
+```json
+{
+  "name"      : "BlockStoragePolicies",
+  "properties":
+  {
+    "BlockStoragePolicies":
+    {
+      "type"      : "object",
+      "properties":
+      {
+        "BlockStoragePolicy":
+        {
+          "description": "An array of BlockStoragePolicy",
+          "type"       : "array",
+          "items"      : blockStoragePolicyProperties      //See BlockStoragePolicy Properties
+        }
+      }
+    }
+  }
+}
+```
 
 HTTP Query Parameter Dictionary
 -------------------------------
@@ -2281,3 +2518,27 @@ See also: [Authentication](#Authentication)
 | Syntax | true |
 
 See also: [Create and Write to a File](#Create_and_Write_to_a_File)
+
+### Storage Policy
+
+| Name | `storagepolicy` |
+|:---- |:---- |
+| Description | The name of the storage policy. |
+| Type | String |
+| Default Value | \<empty\> |
+| Valid Values | Any valid storage policy name; see [GETALLSTORAGEPOLICY](#Get_all_Storage_Policies).  |
+| Syntax | Any string. |
+
+See also: [`SETSTORAGEPOLICY`](#Set_Storage_Policy)
+
+### Start After
+
+| Name | `startAfter` |
+|:---- |:---- |
+| Description | The last item returned in the liststatus batch. |
+| Type | String |
+| Default Value | \<empty\> |
+| Valid Values | Any valid file/directory name. |
+| Syntax | Any string. |
+
+See also: [`LISTSTATUS_BATCH`](#Iteratively_List_a_Directory)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 5386a45..259353c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -37,6 +37,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
 import java.util.Random;
 
 import org.apache.commons.io.IOUtils;
@@ -44,6 +45,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -58,10 +60,13 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestDFSClientRetries;
 import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -1113,4 +1118,67 @@ public class TestWebHDFS {
       }
     }
   }
+
+
+  @Test
+  public void testStoragePolicy() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    final Path path = new Path("/file");
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final WebHdfsFileSystem webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+          conf, WebHdfsConstants.WEBHDFS_SCHEME);
+
+      // test getAllStoragePolicies
+      BlockStoragePolicy[] dfsPolicies = (BlockStoragePolicy[]) dfs
+          .getAllStoragePolicies().toArray();
+      BlockStoragePolicy[] webHdfsPolicies = (BlockStoragePolicy[]) webHdfs
+          .getAllStoragePolicies().toArray();
+      Assert.assertTrue(Arrays.equals(dfsPolicies, webHdfsPolicies));
+
+      // test get/set/unset policies
+      DFSTestUtil.createFile(dfs, path, 0, (short) 1, 0L);
+      // get defaultPolicy
+      BlockStoragePolicySpi defaultdfsPolicy = dfs.getStoragePolicy(path);
+      // set policy through webhdfs
+      webHdfs.setStoragePolicy(path, HdfsConstants.COLD_STORAGE_POLICY_NAME);
+      // get policy from dfs
+      BlockStoragePolicySpi dfsPolicy = dfs.getStoragePolicy(path);
+      // get policy from webhdfs
+      BlockStoragePolicySpi webHdfsPolicy = webHdfs.getStoragePolicy(path);
+      Assert.assertEquals(HdfsConstants.COLD_STORAGE_POLICY_NAME.toString(),
+          webHdfsPolicy.getName());
+      Assert.assertEquals(webHdfsPolicy, dfsPolicy);
+      // unset policy
+      webHdfs.unsetStoragePolicy(path);
+      Assert.assertEquals(defaultdfsPolicy, webHdfs.getStoragePolicy(path));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testSetStoragePolicyWhenPolicyDisabled() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+        .build();
+    try {
+      cluster.waitActive();
+      final WebHdfsFileSystem webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+          conf, WebHdfsConstants.WEBHDFS_SCHEME);
+      webHdfs.setStoragePolicy(new Path("/"),
+          HdfsConstants.COLD_STORAGE_POLICY_NAME);
+      fail("Should throw exception, when storage policy disabled");
+    } catch (IOException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Failed to set storage policy since"));
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2895f4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
index fce8917..6449bf7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
@@ -461,4 +461,12 @@ public class TestParam {
     StartAfterParam param = new StartAfterParam(s);
     Assert.assertEquals(s, param.getValue());
   }
+
+  @Test
+  public void testStoragePolicyParam() {
+    StoragePolicyParam p = new StoragePolicyParam(StoragePolicyParam.DEFAULT);
+    Assert.assertEquals(null, p.getValue());
+    p = new StoragePolicyParam("COLD");
+    Assert.assertEquals("COLD", p.getValue());
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/50] [abbrv] hadoop git commit: HDFS-11211. Add a time unit to the DataNode client trace format. Contributed by Jagadesh Kiran N

Posted by st...@apache.org.
HDFS-11211. Add a time unit to the DataNode client trace format. Contributed by Jagadesh Kiran N


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

Branch: refs/heads/HADOOP-13345
Commit: 43cb1678cc000b1fbf8b813ee14ecef19b86a55c
Parents: ed89856
Author: Mingliang Liu <li...@apache.org>
Authored: Tue Dec 6 09:12:11 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Dec 6 09:12:11 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43cb1678/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 22a70fa..b845da0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -273,7 +273,7 @@ public class DataNode extends ReconfigurableBase
         ", offset: %s" + // offset
         ", srvID: %s" +  // DatanodeRegistration
         ", blockid: %s" + // block id
-        ", duration: %s";  // duration time
+        ", duration(ns): %s";  // duration time
         
   static final Log ClientTraceLog =
     LogFactory.getLog(DataNode.class.getName() + ".clienttrace");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/50] [abbrv] hadoop git commit: HDFS-11181. Fuse wrapper has a typo. Contributed by Wei-Chiu Chuang.

Posted by st...@apache.org.
HDFS-11181. Fuse wrapper has a typo. Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/HADOOP-13345
Commit: c51bfd29cd1e6ec619742f2c47ebfc8bbfb231b6
Parents: f885160
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Mon Dec 5 08:44:40 2016 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Mon Dec 5 08:44:40 2016 -0800

----------------------------------------------------------------------
 .../src/main/native/fuse-dfs/fuse_dfs_wrapper.sh                   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c51bfd29/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
index c52c5f9..d5bfd09 100755
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs_wrapper.sh
@@ -43,7 +43,7 @@ done < <(find "$HADOOP_HOME/hadoop-client" -name "*.jar" -print0)
 while IFS= read -r -d '' file
 do
   export CLASSPATH=$CLASSPATH:$file
-done < <(find "$HADOOP_HOME/hhadoop-hdfs-project" -name "*.jar" -print0)
+done < <(find "$HADOOP_HOME/hadoop-hdfs-project" -name "*.jar" -print0)
 
 export CLASSPATH=$HADOOP_CONF_DIR:$CLASSPATH
 export PATH=$FUSEDFS_PATH:$PATH


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/50] [abbrv] hadoop git commit: HDFS-10930. Refactor: Wrap Datanode IO related operations. Contributed by Xiaoyu Yao.

Posted by st...@apache.org.
HDFS-10930. Refactor: Wrap Datanode IO related operations. Contributed by Xiaoyu Yao.


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

Branch: refs/heads/HADOOP-13345
Commit: df983b524ab68ea0c70cee9033bfff2d28052cbf
Parents: 43cb167
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Mon Dec 5 13:04:39 2016 -0800
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Dec 6 11:05:47 2016 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockReceiver.java     |  66 +++----
 .../hdfs/server/datanode/BlockSender.java       | 105 ++++-------
 .../hadoop/hdfs/server/datanode/DNConf.java     |   4 +
 .../hdfs/server/datanode/DataStorage.java       |   5 +
 .../hdfs/server/datanode/LocalReplica.java      | 179 +++++++++++++------
 .../server/datanode/LocalReplicaInPipeline.java |  30 ++--
 .../hdfs/server/datanode/ReplicaInPipeline.java |   4 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   3 +-
 .../datanode/fsdataset/ReplicaInputStreams.java | 102 ++++++++++-
 .../fsdataset/ReplicaOutputStreams.java         | 107 ++++++++++-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |  97 +++++-----
 .../impl/FsDatasetAsyncDiskService.java         |   7 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   5 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   5 +-
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |   2 +-
 .../server/datanode/SimulatedFSDataset.java     |   8 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   2 +-
 .../server/datanode/TestSimulatedFSDataset.java |   2 +-
 .../extdataset/ExternalDatasetImpl.java         |   4 +-
 .../extdataset/ExternalReplicaInPipeline.java   |   6 +-
 20 files changed, 470 insertions(+), 273 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 39419c1..f372072 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -24,10 +24,7 @@ import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
-import java.io.FileDescriptor;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.nio.ByteBuffer;
@@ -53,7 +50,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
@@ -88,8 +84,6 @@ class BlockReceiver implements Closeable {
    * the DataNode needs to recalculate checksums before writing.
    */
   private final boolean needsChecksumTranslation;
-  private OutputStream out = null; // to block file at local disk
-  private FileDescriptor outFd;
   private DataOutputStream checksumOut = null; // to crc file at local disk
   private final int bytesPerChecksum;
   private final int checksumSize;
@@ -250,7 +244,8 @@ class BlockReceiver implements Closeable {
       
       final boolean isCreate = isDatanode || isTransfer 
           || stage == BlockConstructionStage.PIPELINE_SETUP_CREATE;
-      streams = replicaInfo.createStreams(isCreate, requestedChecksum);
+      streams = replicaInfo.createStreams(isCreate, requestedChecksum,
+          datanodeSlowLogThresholdMs);
       assert streams != null : "null streams!";
 
       // read checksum meta information
@@ -260,13 +255,6 @@ class BlockReceiver implements Closeable {
       this.bytesPerChecksum = diskChecksum.getBytesPerChecksum();
       this.checksumSize = diskChecksum.getChecksumSize();
 
-      this.out = streams.getDataOut();
-      if (out instanceof FileOutputStream) {
-        this.outFd = ((FileOutputStream)out).getFD();
-      } else {
-        LOG.warn("Could not get file descriptor for outputstream of class " +
-            out.getClass());
-      }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
           streams.getChecksumOut(), DFSUtilClient.getSmallBufferSize(
           datanode.getConf())));
@@ -319,7 +307,7 @@ class BlockReceiver implements Closeable {
     packetReceiver.close();
 
     IOException ioe = null;
-    if (syncOnClose && (out != null || checksumOut != null)) {
+    if (syncOnClose && (streams.getDataOut() != null || checksumOut != null)) {
       datanode.metrics.incrFsyncCount();      
     }
     long flushTotalNanos = 0;
@@ -348,9 +336,9 @@ class BlockReceiver implements Closeable {
     }
     // close block file
     try {
-      if (out != null) {
+      if (streams.getDataOut() != null) {
         long flushStartNanos = System.nanoTime();
-        out.flush();
+        streams.flushDataOut();
         long flushEndNanos = System.nanoTime();
         if (syncOnClose) {
           long fsyncStartNanos = flushEndNanos;
@@ -359,14 +347,13 @@ class BlockReceiver implements Closeable {
         }
         flushTotalNanos += flushEndNanos - flushStartNanos;
         measuredFlushTime = true;
-        out.close();
-        out = null;
+        streams.closeDataStream();
       }
     } catch (IOException e) {
       ioe = e;
     }
     finally{
-      IOUtils.closeStream(out);
+      streams.close();
     }
     if (replicaHandler != null) {
       IOUtils.cleanup(null, replicaHandler);
@@ -419,9 +406,9 @@ class BlockReceiver implements Closeable {
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
-    if (out != null) {
+    if (streams.getDataOut() != null) {
       long flushStartNanos = System.nanoTime();
-      out.flush();
+      streams.flushDataOut();
       long flushEndNanos = System.nanoTime();
       if (isSync) {
         long fsyncStartNanos = flushEndNanos;
@@ -430,10 +417,10 @@ class BlockReceiver implements Closeable {
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
-    if (checksumOut != null || out != null) {
+    if (checksumOut != null || streams.getDataOut() != null) {
       datanode.metrics.addFlushNanos(flushTotalNanos);
       if (isSync) {
-    	  datanode.metrics.incrFsyncCount();      
+        datanode.metrics.incrFsyncCount();
       }
     }
     long duration = Time.monotonicNow() - begin;
@@ -716,16 +703,12 @@ class BlockReceiver implements Closeable {
           int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
           
           // Write data to disk.
-          long begin = Time.monotonicNow();
-          out.write(dataBuf.array(), startByteToDisk, numBytesToDisk);
-          long duration = Time.monotonicNow() - begin;
+          long duration = streams.writeToDisk(dataBuf.array(),
+              startByteToDisk, numBytesToDisk);
+
           if (duration > maxWriteToDiskMs) {
             maxWriteToDiskMs = duration;
           }
-          if (duration > datanodeSlowLogThresholdMs) {
-            LOG.warn("Slow BlockReceiver write data to disk cost:" + duration
-                + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
-          }
 
           final byte[] lastCrc;
           if (shouldNotWriteChecksum) {
@@ -842,7 +825,7 @@ class BlockReceiver implements Closeable {
 
   private void manageWriterOsCache(long offsetInBlock) {
     try {
-      if (outFd != null &&
+      if (streams.getOutFd() != null &&
           offsetInBlock > lastCacheManagementOffset + CACHE_DROP_LAG_BYTES) {
         long begin = Time.monotonicNow();
         //
@@ -857,12 +840,11 @@ class BlockReceiver implements Closeable {
         if (syncBehindWrites) {
           if (syncBehindWritesInBackground) {
             this.datanode.getFSDataset().submitBackgroundSyncFileRangeRequest(
-                block, outFd, lastCacheManagementOffset,
+                block, streams, lastCacheManagementOffset,
                 offsetInBlock - lastCacheManagementOffset,
                 SYNC_FILE_RANGE_WRITE);
           } else {
-            NativeIO.POSIX.syncFileRangeIfPossible(outFd,
-                lastCacheManagementOffset,
+            streams.syncFileRangeIfPossible(lastCacheManagementOffset,
                 offsetInBlock - lastCacheManagementOffset,
                 SYNC_FILE_RANGE_WRITE);
           }
@@ -879,8 +861,8 @@ class BlockReceiver implements Closeable {
         //                     
         long dropPos = lastCacheManagementOffset - CACHE_DROP_LAG_BYTES;
         if (dropPos > 0 && dropCacheBehindWrites) {
-          NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-              block.getBlockName(), outFd, 0, dropPos, POSIX_FADV_DONTNEED);
+          streams.dropCacheBehindWrites(block.getBlockName(), 0, dropPos,
+              POSIX_FADV_DONTNEED);
         }
         lastCacheManagementOffset = offsetInBlock;
         long duration = Time.monotonicNow() - begin;
@@ -989,7 +971,7 @@ class BlockReceiver implements Closeable {
               // The worst case is not recovering this RBW replica. 
               // Client will fall back to regular pipeline recovery.
             } finally {
-              IOUtils.closeStream(out);
+              IOUtils.closeStream(streams.getDataOut());
             }
             try {              
               // Even if the connection is closed after the ack packet is
@@ -1047,8 +1029,8 @@ class BlockReceiver implements Closeable {
    * will be overwritten.
    */
   private void adjustCrcFilePosition() throws IOException {
-    if (out != null) {
-     out.flush();
+    if (streams.getDataOut() != null) {
+      streams.flushDataOut();
     }
     if (checksumOut != null) {
       checksumOut.flush();
@@ -1094,10 +1076,10 @@ class BlockReceiver implements Closeable {
     byte[] crcbuf = new byte[checksumSize];
     try (ReplicaInputStreams instr =
         datanode.data.getTmpInputStreams(block, blkoff, ckoff)) {
-      IOUtils.readFully(instr.getDataIn(), buf, 0, sizePartialChunk);
+      instr.readDataFully(buf, 0, sizePartialChunk);
 
       // open meta file and read in crc value computer earlier
-      IOUtils.readFully(instr.getChecksumIn(), crcbuf, 0, crcbuf.length);
+      instr.readChecksumFully(crcbuf, 0, crcbuf.length);
     }
 
     // compute crc of partial chunk from data read in the block file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index a1b1f86..9182c88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
-import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -42,11 +41,11 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
@@ -120,12 +119,11 @@ class BlockSender implements java.io.Closeable {
   
   /** the block to read from */
   private final ExtendedBlock block;
-  /** Stream to read block data from */
-  private InputStream blockIn;
+
+  /** InputStreams and file descriptors to read block/checksum. */
+  private ReplicaInputStreams ris;
   /** updated while using transferTo() */
   private long blockInPosition = -1;
-  /** Stream to read checksum */
-  private DataInputStream checksumIn;
   /** Checksum utility */
   private final DataChecksum checksum;
   /** Initial position to read */
@@ -152,11 +150,6 @@ class BlockSender implements java.io.Closeable {
   private final String clientTraceFmt;
   private volatile ChunkChecksum lastChunkChecksum = null;
   private DataNode datanode;
-  
-  /** The file descriptor of the block being sent */
-  private FileDescriptor blockInFd;
-  /** The reference to the volume where the block is located */
-  private FsVolumeReference volumeRef;
 
   /** The replica of the block that is being read. */
   private final Replica replica;
@@ -201,6 +194,9 @@ class BlockSender implements java.io.Closeable {
               boolean sendChecksum, DataNode datanode, String clientTraceFmt,
               CachingStrategy cachingStrategy)
       throws IOException {
+    InputStream blockIn = null;
+    DataInputStream checksumIn = null;
+    FsVolumeReference volumeRef = null;
     try {
       this.block = block;
       this.corruptChecksumOk = corruptChecksumOk;
@@ -281,7 +277,7 @@ class BlockSender implements java.io.Closeable {
         (!is32Bit || length <= Integer.MAX_VALUE);
 
       // Obtain a reference before reading data
-      this.volumeRef = datanode.data.getVolume(block).obtainReference();
+      volumeRef = datanode.data.getVolume(block).obtainReference();
 
       /* 
        * (corruptChecksumOK, meta_file_exist): operation
@@ -405,14 +401,9 @@ class BlockSender implements java.io.Closeable {
         DataNode.LOG.debug("replica=" + replica);
       }
       blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
-      if (blockIn instanceof FileInputStream) {
-        blockInFd = ((FileInputStream)blockIn).getFD();
-      } else {
-        blockInFd = null;
-      }
+      ris = new ReplicaInputStreams(blockIn, checksumIn, volumeRef);
     } catch (IOException ioe) {
       IOUtils.closeStream(this);
-      IOUtils.closeStream(blockIn);
       throw ioe;
     }
   }
@@ -422,12 +413,11 @@ class BlockSender implements java.io.Closeable {
    */
   @Override
   public void close() throws IOException {
-    if (blockInFd != null &&
+    if (ris.getDataInFd() != null &&
         ((dropCacheBehindAllReads) ||
          (dropCacheBehindLargeReads && isLongRead()))) {
       try {
-        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-            block.getBlockName(), blockInFd, lastCacheDropOffset,
+        ris.dropCacheBehindReads(block.getBlockName(), lastCacheDropOffset,
             offset - lastCacheDropOffset, POSIX_FADV_DONTNEED);
       } catch (Exception e) {
         LOG.warn("Unable to drop cache on file close", e);
@@ -436,32 +426,12 @@ class BlockSender implements java.io.Closeable {
     if (curReadahead != null) {
       curReadahead.cancel();
     }
-    
-    IOException ioe = null;
-    if(checksumIn!=null) {
-      try {
-        checksumIn.close(); // close checksum file
-      } catch (IOException e) {
-        ioe = e;
-      }
-      checksumIn = null;
-    }   
-    if(blockIn!=null) {
-      try {
-        blockIn.close(); // close data file
-      } catch (IOException e) {
-        ioe = e;
-      }
-      blockIn = null;
-      blockInFd = null;
-    }
-    if (volumeRef != null) {
-      IOUtils.cleanup(null, volumeRef);
-      volumeRef = null;
-    }
-    // throw IOException if there is any
-    if(ioe!= null) {
-      throw ioe;
+
+    try {
+      ris.closeStreams();
+    } finally {
+      IOUtils.closeStream(ris);
+      ris = null;
     }
   }
   
@@ -565,7 +535,7 @@ class BlockSender implements java.io.Closeable {
     int checksumOff = pkt.position();
     byte[] buf = pkt.array();
     
-    if (checksumSize > 0 && checksumIn != null) {
+    if (checksumSize > 0 && ris.getChecksumIn() != null) {
       readChecksum(buf, checksumOff, checksumDataLen);
 
       // write in progress that we need to use to get last checksum
@@ -581,7 +551,7 @@ class BlockSender implements java.io.Closeable {
     
     int dataOff = checksumOff + checksumDataLen;
     if (!transferTo) { // normal transfer
-      IOUtils.readFully(blockIn, buf, dataOff, dataLen);
+      ris.readDataFully(buf, dataOff, dataLen);
 
       if (verifyChecksum) {
         verifyChecksum(buf, dataOff, dataLen, numChunks, checksumOff);
@@ -593,12 +563,12 @@ class BlockSender implements java.io.Closeable {
         SocketOutputStream sockOut = (SocketOutputStream)out;
         // First write header and checksums
         sockOut.write(buf, headerOff, dataOff - headerOff);
-        
+
         // no need to flush since we know out is not a buffered stream
-        FileChannel fileCh = ((FileInputStream)blockIn).getChannel();
+        FileChannel fileCh = ((FileInputStream)ris.getDataIn()).getChannel();
         LongWritable waitTime = new LongWritable();
         LongWritable transferTime = new LongWritable();
-        sockOut.transferToFully(fileCh, blockInPosition, dataLen, 
+        sockOut.transferToFully(fileCh, blockInPosition, dataLen,
             waitTime, transferTime);
         datanode.metrics.addSendDataPacketBlockedOnNetworkNanos(waitTime.get());
         datanode.metrics.addSendDataPacketTransferNanos(transferTime.get());
@@ -630,7 +600,7 @@ class BlockSender implements java.io.Closeable {
         if (!ioem.startsWith("Broken pipe") && !ioem.startsWith("Connection reset")) {
           LOG.error("BlockSender.sendChunks() exception: ", e);
           datanode.getBlockScanner().markSuspectBlock(
-              volumeRef.getVolume().getStorageID(),
+              ris.getVolumeRef().getVolume().getStorageID(),
               block);
         }
       }
@@ -653,16 +623,15 @@ class BlockSender implements java.io.Closeable {
    */
   private void readChecksum(byte[] buf, final int checksumOffset,
       final int checksumLen) throws IOException {
-    if (checksumSize <= 0 && checksumIn == null) {
+    if (checksumSize <= 0 && ris.getChecksumIn() == null) {
       return;
     }
     try {
-      checksumIn.readFully(buf, checksumOffset, checksumLen);
+      ris.readChecksumFully(buf, checksumOffset, checksumLen);
     } catch (IOException e) {
       LOG.warn(" Could not read or failed to verify checksum for data"
           + " at offset " + offset + " for block " + block, e);
-      IOUtils.closeStream(checksumIn);
-      checksumIn = null;
+      ris.closeChecksumStream();
       if (corruptChecksumOk) {
         if (checksumOffset < checksumLen) {
           // Just fill the array with zeros.
@@ -746,10 +715,10 @@ class BlockSender implements java.io.Closeable {
     
     lastCacheDropOffset = initialOffset;
 
-    if (isLongRead() && blockInFd != null) {
+    if (isLongRead() && ris.getDataInFd() != null) {
       // Advise that this file descriptor will be accessed sequentially.
-      NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-          block.getBlockName(), blockInFd, 0, 0, POSIX_FADV_SEQUENTIAL);
+      ris.dropCacheBehindReads(block.getBlockName(), 0, 0,
+          POSIX_FADV_SEQUENTIAL);
     }
     
     // Trigger readahead of beginning of file if configured.
@@ -761,9 +730,10 @@ class BlockSender implements java.io.Closeable {
       int pktBufSize = PacketHeader.PKT_MAX_HEADER_LEN;
       boolean transferTo = transferToAllowed && !verifyChecksum
           && baseStream instanceof SocketOutputStream
-          && blockIn instanceof FileInputStream;
+          && ris.getDataIn() instanceof FileInputStream;
       if (transferTo) {
-        FileChannel fileChannel = ((FileInputStream)blockIn).getChannel();
+        FileChannel fileChannel =
+            ((FileInputStream)ris.getDataIn()).getChannel();
         blockInPosition = fileChannel.position();
         streamForSendChunks = baseStream;
         maxChunksPerPacket = numberOfChunks(TRANSFERTO_BUFFER_SIZE);
@@ -818,14 +788,16 @@ class BlockSender implements java.io.Closeable {
   private void manageOsCache() throws IOException {
     // We can't manage the cache for this block if we don't have a file
     // descriptor to work with.
-    if (blockInFd == null) return;
+    if (ris.getDataInFd() == null) {
+      return;
+    }
 
     // Perform readahead if necessary
     if ((readaheadLength > 0) && (datanode.readaheadPool != null) &&
           (alwaysReadahead || isLongRead())) {
       curReadahead = datanode.readaheadPool.readaheadStream(
-          clientTraceFmt, blockInFd, offset, readaheadLength, Long.MAX_VALUE,
-          curReadahead);
+          clientTraceFmt, ris.getDataInFd(), offset, readaheadLength,
+          Long.MAX_VALUE, curReadahead);
     }
 
     // Drop what we've just read from cache, since we aren't
@@ -835,8 +807,7 @@ class BlockSender implements java.io.Closeable {
       long nextCacheDropOffset = lastCacheDropOffset + CACHE_DROP_INTERVAL_BYTES;
       if (offset >= nextCacheDropOffset) {
         long dropLength = offset - lastCacheDropOffset;
-        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-            block.getBlockName(), blockInFd, lastCacheDropOffset,
+        ris.dropCacheBehindReads(block.getBlockName(), lastCacheDropOffset,
             dropLength, POSIX_FADV_DONTNEED);
         lastCacheDropOffset = offset;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index 823d05c..c1487b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -402,6 +402,10 @@ public class DNConf {
     return volsConfigured;
   }
 
+  public long getSlowIoWarningThresholdMs() {
+    return datanodeSlowIoWarningThresholdMs;
+  }
+
   int getMaxDataLength() {
     return maxDataLength;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 29b14e7..f4deb6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -1355,4 +1355,9 @@ public class DataStorage extends Storage {
   synchronized void removeBlockPoolStorage(String bpId) {
     bpStorageMap.remove(bpId);
   }
+
+  public static boolean fullyDelete(final File dir) {
+    boolean result = FileUtil.fullyDelete(dir);
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
index f829111..e6f7e12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
@@ -29,9 +29,6 @@ import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -46,6 +43,8 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -69,15 +68,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   private static final Map<String, File> internedBaseDirs = new HashMap<String, File>();
 
-  static final Log LOG = LogFactory.getLog(LocalReplica.class);
-  private final static boolean IS_NATIVE_IO_AVAIL;
-  static {
-    IS_NATIVE_IO_AVAIL = NativeIO.isAvailable();
-    if (Path.WINDOWS && !IS_NATIVE_IO_AVAIL) {
-      LOG.warn("Data node cannot fully support concurrent reading"
-          + " and writing without native code extensions on Windows.");
-    }
-  }
+  static final Logger LOG = LoggerFactory.getLogger(LocalReplica.class);
 
   /**
    * Constructor
@@ -199,14 +190,14 @@ abstract public class LocalReplica extends ReplicaInfo {
     File tmpFile = DatanodeUtil.createTmpFile(b, DatanodeUtil.getUnlinkTmpFile(file));
     try (FileInputStream in = new FileInputStream(file)) {
       try (FileOutputStream out = new FileOutputStream(tmpFile)){
-        IOUtils.copyBytes(in, out, 16 * 1024);
+        copyBytes(in, out, 16 * 1024);
       }
       if (file.length() != tmpFile.length()) {
         throw new IOException("Copy of file " + file + " size " + file.length()+
                               " into file " + tmpFile +
                               " resulted in a size of " + tmpFile.length());
       }
-      FileUtil.replaceFile(tmpFile, file);
+      replaceFile(tmpFile, file);
     } catch (IOException e) {
       boolean done = tmpFile.delete();
       if (!done) {
@@ -241,13 +232,13 @@ abstract public class LocalReplica extends ReplicaInfo {
     }
     File meta = getMetaFile();
 
-    int linkCount = HardLink.getLinkCount(file);
+    int linkCount = getHardLinkCount(file);
     if (linkCount > 1) {
       DataNode.LOG.info("Breaking hardlink for " + linkCount + "x-linked " +
           "block " + this);
       breakHardlinks(file, this);
     }
-    if (HardLink.getLinkCount(meta) > 1) {
+    if (getHardLinkCount(meta) > 1) {
       breakHardlinks(meta, this);
     }
     return true;
@@ -260,18 +251,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public InputStream getDataInputStream(long seekOffset) throws IOException {
-
-    File blockFile = getBlockFile();
-    if (IS_NATIVE_IO_AVAIL) {
-      return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
-    } else {
-      try {
-        return FsDatasetUtil.openAndSeek(blockFile, seekOffset);
-      } catch (FileNotFoundException fnfe) {
-        throw new IOException("Block " + this + " is not valid. " +
-            "Expected block file at " + blockFile + " does not exist.");
-      }
-    }
+    return getDataInputStream(getBlockFile(), seekOffset);
   }
 
   @Override
@@ -286,7 +266,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean deleteBlockData() {
-    return getBlockFile().delete();
+    return fullyDelete(getBlockFile());
   }
 
   @Override
@@ -320,7 +300,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean deleteMetadata() {
-    return getMetaFile().delete();
+    return fullyDelete(getMetaFile());
   }
 
   @Override
@@ -340,7 +320,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   private boolean renameFile(File srcfile, File destfile) throws IOException {
     try {
-      NativeIO.renameTo(srcfile, destfile);
+      rename(srcfile, destfile);
       return true;
     } catch (IOException e) {
       throw new IOException("Failed to move block file for " + this
@@ -367,22 +347,14 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean getPinning(LocalFileSystem localFS) throws IOException {
-    FileStatus fss =
-        localFS.getFileStatus(new Path(getBlockFile().getAbsolutePath()));
-    return fss.getPermission().getStickyBit();
+    return getPinning(localFS, new Path(getBlockFile().getAbsolutePath()));
   }
 
   @Override
   public void setPinning(LocalFileSystem localFS) throws IOException {
     File f = getBlockFile();
     Path p = new Path(f.getAbsolutePath());
-
-    FsPermission oldPermission = localFS.getFileStatus(
-        new Path(f.getAbsolutePath())).getPermission();
-    //sticky bit is used for pinning purpose
-    FsPermission permission = new FsPermission(oldPermission.getUserAction(),
-        oldPermission.getGroupAction(), oldPermission.getOtherAction(), true);
-    localFS.setPermission(p, permission);
+    setPinning(localFS, p);
   }
 
   @Override
@@ -398,7 +370,7 @@ abstract public class LocalReplica extends ReplicaInfo {
     }
     try {
       // calling renameMeta on the ReplicaInfo doesn't work here
-      NativeIO.renameTo(oldmeta, newmeta);
+      rename(oldmeta, newmeta);
     } catch (IOException e) {
       setGenerationStamp(oldGS); // restore old GS
       throw new IOException("Block " + this + " reopen failed. " +
@@ -417,7 +389,113 @@ abstract public class LocalReplica extends ReplicaInfo {
     return info.getBlockFile().compareTo(getBlockFile());
   }
 
-  static public void truncateBlock(File blockFile, File metaFile,
+  @Override
+  public void copyMetadata(URI destination) throws IOException {
+    //for local replicas, we assume the destination URI is file
+    nativeCopyFileUnbuffered(getMetaFile(), new File(destination), true);
+  }
+
+  @Override
+  public void copyBlockdata(URI destination) throws IOException {
+    //for local replicas, we assume the destination URI is file
+    nativeCopyFileUnbuffered(getBlockFile(), new File(destination), true);
+  }
+
+  public void renameMeta(File newMetaFile) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming " + getMetaFile() + " to " + newMetaFile);
+    }
+    renameFile(getMetaFile(), newMetaFile);
+  }
+
+  public void renameBlock(File newBlockFile) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming " + getBlockFile() + " to " + newBlockFile
+          + ", file length=" + getBlockFile().length());
+    }
+    renameFile(getBlockFile(), newBlockFile);
+  }
+
+  public static void rename(File from, File to) throws IOException {
+    Storage.rename(from, to);
+  }
+
+  /**
+   * Get input stream for a local file and optionally seek to the offset.
+   * @param f path to the file
+   * @param seekOffset offset to seek
+   * @return input stream for read
+   * @throws IOException
+   */
+  private FileInputStream getDataInputStream(File f, long seekOffset)
+      throws IOException {
+    FileInputStream fis;
+    if (NativeIO.isAvailable()) {
+      fis = NativeIO.getShareDeleteFileInputStream(f, seekOffset);
+    } else {
+      try {
+        fis = FsDatasetUtil.openAndSeek(f, seekOffset);
+      } catch (FileNotFoundException fnfe) {
+        throw new IOException("Expected block file at " + f +
+            " does not exist.");
+      }
+    }
+    return fis;
+  }
+
+  private void nativeCopyFileUnbuffered(File srcFile, File destFile,
+      boolean preserveFileDate) throws IOException {
+    Storage.nativeCopyFileUnbuffered(srcFile, destFile, preserveFileDate);
+  }
+
+  private void copyBytes(InputStream in, OutputStream out, int
+      buffSize) throws IOException{
+    IOUtils.copyBytes(in, out, buffSize);
+  }
+
+  private void replaceFile(File src, File target) throws IOException {
+    FileUtil.replaceFile(src, target);
+  }
+
+  public static boolean fullyDelete(final File dir) {
+    boolean result = DataStorage.fullyDelete(dir);
+    return result;
+  }
+
+  public static int getHardLinkCount(File fileName) throws IOException {
+    int linkCount = HardLink.getLinkCount(fileName);
+    return linkCount;
+  }
+
+  /**
+   *  Get pin status of a file by checking the sticky bit.
+   * @param localFS local file system
+   * @param path path to be checked
+   * @return true if the file is pinned with sticky bit
+   * @throws IOException
+   */
+  public boolean getPinning(LocalFileSystem localFS, Path path) throws
+      IOException {
+    boolean stickyBit =
+        localFS.getFileStatus(path).getPermission().getStickyBit();
+    return stickyBit;
+  }
+
+  /**
+   * Set sticky bit on path to pin file.
+   * @param localFS local file system
+   * @param path path to be pinned with sticky bit
+   * @throws IOException
+   */
+  public void setPinning(LocalFileSystem localFS, Path path) throws
+      IOException {
+    FsPermission oldPermission = localFS.getFileStatus(path).getPermission();
+    FsPermission permission = new FsPermission(oldPermission.getUserAction(),
+        oldPermission.getGroupAction(), oldPermission.getOtherAction(), true);
+    localFS.setPermission(path, permission);
+  }
+
+  public static void truncateBlock(File blockFile, File metaFile,
       long oldlen, long newlen) throws IOException {
     LOG.info("truncateBlock: blockFile=" + blockFile
         + ", metaFile=" + metaFile
@@ -467,19 +545,4 @@ abstract public class LocalReplica extends ReplicaInfo {
       metaRAF.close();
     }
   }
-
-  @Override
-  public void copyMetadata(URI destination) throws IOException {
-    //for local replicas, we assume the destination URI is file
-    Storage.nativeCopyFileUnbuffered(getMetaFile(),
-        new File(destination), true);
-  }
-
-  @Override
-  public void copyBlockdata(URI destination) throws IOException {
-    //for local replicas, we assume the destination URI is file
-    Storage.nativeCopyFileUnbuffered(getBlockFile(),
-        new File(destination), true);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
index bc7bc6d..1387155 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 
@@ -246,7 +245,8 @@ public class LocalReplicaInPipeline extends LocalReplica
 
   @Override // ReplicaInPipeline
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum) throws IOException {
+      DataChecksum requestedChecksum, long slowLogThresholdMs)
+      throws IOException {
     File blockFile = getBlockFile();
     File metaFile = getMetaFile();
     if (DataNode.LOG.isDebugEnabled()) {
@@ -313,7 +313,7 @@ public class LocalReplicaInPipeline extends LocalReplica
         crcOut.getChannel().position(crcDiskSize);
       }
       return new ReplicaOutputStreams(blockOut, crcOut, checksum,
-          getVolume().isTransientStorage());
+          getVolume().isTransientStorage(), slowLogThresholdMs);
     } catch (IOException e) {
       IOUtils.closeStream(blockOut);
       IOUtils.closeStream(metaRAF);
@@ -373,40 +373,30 @@ public class LocalReplicaInPipeline extends LocalReplica
           + " should be derived from LocalReplica");
     }
 
-    LocalReplica localReplica = (LocalReplica) oldReplicaInfo;
-
-    File oldmeta = localReplica.getMetaFile();
+    LocalReplica oldReplica = (LocalReplica) oldReplicaInfo;
+    File oldmeta = oldReplica.getMetaFile();
     File newmeta = getMetaFile();
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + oldmeta + " to " + newmeta);
-    }
     try {
-      NativeIO.renameTo(oldmeta, newmeta);
+      oldReplica.renameMeta(newmeta);
     } catch (IOException e) {
       throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
                             " Unable to move meta file  " + oldmeta +
                             " to rbw dir " + newmeta, e);
     }
 
-    File blkfile = localReplica.getBlockFile();
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + blkfile + " to " + newBlkFile
-          + ", file length=" + blkfile.length());
-    }
     try {
-      NativeIO.renameTo(blkfile, newBlkFile);
+      oldReplica.renameBlock(newBlkFile);
     } catch (IOException e) {
       try {
-        NativeIO.renameTo(newmeta, oldmeta);
+        renameMeta(oldmeta);
       } catch (IOException ex) {
         LOG.warn("Cannot move meta file " + newmeta +
             "back to the finalized directory " + oldmeta, ex);
       }
       throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
-                              " Unable to move block file " + blkfile +
-                              " to rbw dir " + newBlkFile, e);
+          " Unable to move block file " + oldReplica.getBlockFile() +
+          " to rbw dir " + newBlkFile, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
index efa6ea6..5fdbec0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
@@ -69,11 +69,13 @@ public interface ReplicaInPipeline extends Replica {
    *
    * @param isCreate if it is for creation
    * @param requestedChecksum the checksum the writer would prefer to use
+   * @param slowLogThresholdMs slow io threshold for logging
    * @return output streams for writing
    * @throws IOException if any error occurs
    */
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum) throws IOException;
+      DataChecksum requestedChecksum, long slowLogThresholdMs)
+      throws IOException;
 
   /**
    * Create an output stream to write restart metadata in case of datanode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 57ec2b4..30f045f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -605,7 +604,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * submit a sync_file_range request to AsyncDiskService.
    */
   void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
-      final FileDescriptor fd, final long offset, final long nbytes,
+      final ReplicaOutputStreams outs, final long offset, final long nbytes,
       final int flags);
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
index 227179d..54d0e96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
@@ -18,24 +18,45 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 import java.io.Closeable;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
 import java.io.InputStream;
+import java.io.IOException;
 
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIOException;
+import org.slf4j.Logger;
 
 /**
  * Contains the input streams for the data and checksum of a replica.
  */
 public class ReplicaInputStreams implements Closeable {
-  private final InputStream dataIn;
-  private final InputStream checksumIn;
-  private final FsVolumeReference volumeRef;
+  public static final Logger LOG = DataNode.LOG;
+
+  private InputStream dataIn;
+  private InputStream checksumIn;
+  private FsVolumeReference volumeRef;
+  private FileDescriptor dataInFd = null;
 
   /** Create an object with a data input stream and a checksum input stream. */
-  public ReplicaInputStreams(InputStream dataStream, InputStream checksumStream,
-      FsVolumeReference volumeRef) {
+  public ReplicaInputStreams(InputStream dataStream,
+      InputStream checksumStream, FsVolumeReference volumeRef) {
     this.volumeRef = volumeRef;
     this.dataIn = dataStream;
     this.checksumIn = checksumStream;
+    if (dataIn instanceof FileInputStream) {
+      try {
+        dataInFd = ((FileInputStream) dataIn).getFD();
+      } catch (Exception e) {
+        LOG.warn("Could not get file descriptor for inputstream of class " +
+            this.dataIn.getClass());
+      }
+    } else {
+      LOG.debug("Could not get file descriptor for inputstream of class " +
+          this.dataIn.getClass());
+    }
   }
 
   /** @return the data input stream. */
@@ -48,10 +69,81 @@ public class ReplicaInputStreams implements Closeable {
     return checksumIn;
   }
 
+  public FileDescriptor getDataInFd() {
+    return dataInFd;
+  }
+
+  public FsVolumeReference getVolumeRef() {
+    return volumeRef;
+  }
+
+  public void readDataFully(byte[] buf, int off, int len)
+      throws IOException {
+    IOUtils.readFully(dataIn, buf, off, len);
+  }
+
+  public void readChecksumFully(byte[] buf, int off, int len)
+      throws IOException {
+    IOUtils.readFully(checksumIn, buf, off, len);
+  }
+
+  public void skipDataFully(long len) throws IOException {
+    IOUtils.skipFully(dataIn, len);
+  }
+
+  public void skipChecksumFully(long len) throws IOException {
+    IOUtils.skipFully(checksumIn, len);
+  }
+
+  public void closeChecksumStream() throws IOException {
+    IOUtils.closeStream(checksumIn);
+    checksumIn = null;
+  }
+
+  public void dropCacheBehindReads(String identifier, long offset, long len,
+      int flags) throws NativeIOException {
+    assert this.dataInFd != null : "null dataInFd!";
+    NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+        identifier, dataInFd, offset, len, flags);
+  }
+
+  public void closeStreams() throws IOException {
+    IOException ioe = null;
+    if(checksumIn!=null) {
+      try {
+        checksumIn.close(); // close checksum file
+      } catch (IOException e) {
+        ioe = e;
+      }
+      checksumIn = null;
+    }
+    if(dataIn!=null) {
+      try {
+        dataIn.close(); // close data file
+      } catch (IOException e) {
+        ioe = e;
+      }
+      dataIn = null;
+      dataInFd = null;
+    }
+    if (volumeRef != null) {
+      IOUtils.cleanup(null, volumeRef);
+      volumeRef = null;
+    }
+    // throw IOException if there is any
+    if(ioe!= null) {
+      throw ioe;
+    }
+  }
+
   @Override
   public void close() {
     IOUtils.closeStream(dataIn);
+    dataIn = null;
+    dataInFd = null;
     IOUtils.closeStream(checksumIn);
+    checksumIn = null;
     IOUtils.cleanup(null, volumeRef);
+    volumeRef = null;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
index bd1461a..a66847a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
@@ -18,32 +18,62 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 import java.io.Closeable;
+import java.io.FileDescriptor;
 import java.io.FileOutputStream;
 import java.io.OutputStream;
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIOException;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
 
 /**
  * Contains the output streams for the data and checksum of a replica.
  */
 public class ReplicaOutputStreams implements Closeable {
-  private final OutputStream dataOut;
+  public static final Logger LOG = DataNode.LOG;
+
+  private FileDescriptor outFd = null;
+  /** Stream to block. */
+  private OutputStream dataOut;
+  /** Stream to checksum. */
   private final OutputStream checksumOut;
   private final DataChecksum checksum;
   private final boolean isTransientStorage;
+  private final long slowLogThresholdMs;
 
   /**
    * Create an object with a data output stream, a checksum output stream
    * and a checksum.
    */
-  public ReplicaOutputStreams(OutputStream dataOut, OutputStream checksumOut,
-      DataChecksum checksum, boolean isTransientStorage) {
+  public ReplicaOutputStreams(OutputStream dataOut,
+      OutputStream checksumOut, DataChecksum checksum,
+      boolean isTransientStorage, long slowLogThresholdMs) {
     this.dataOut = dataOut;
-    this.checksumOut = checksumOut;
     this.checksum = checksum;
+    this.slowLogThresholdMs = slowLogThresholdMs;
     this.isTransientStorage = isTransientStorage;
+    this.checksumOut = checksumOut;
+
+    try {
+      if (this.dataOut instanceof FileOutputStream) {
+        this.outFd = ((FileOutputStream)this.dataOut).getFD();
+      } else {
+        LOG.debug("Could not get file descriptor for outputstream of class " +
+            this.dataOut.getClass());
+      }
+    } catch (IOException e) {
+      LOG.warn("Could not get file descriptor for outputstream of class " +
+          this.dataOut.getClass());
+    }
+  }
+
+  public FileDescriptor getOutFd() {
+    return outFd;
   }
 
   /** @return the data output stream. */
@@ -72,12 +102,17 @@ public class ReplicaOutputStreams implements Closeable {
     IOUtils.closeStream(checksumOut);
   }
 
+  public void closeDataStream() throws IOException {
+    dataOut.close();
+    dataOut = null;
+  }
+
   /**
    * Sync the data stream if it supports it.
    */
   public void syncDataOut() throws IOException {
     if (dataOut instanceof FileOutputStream) {
-      ((FileOutputStream)dataOut).getChannel().force(true);
+      sync((FileOutputStream)dataOut);
     }
   }
   
@@ -86,8 +121,68 @@ public class ReplicaOutputStreams implements Closeable {
    */
   public void syncChecksumOut() throws IOException {
     if (checksumOut instanceof FileOutputStream) {
-      ((FileOutputStream)checksumOut).getChannel().force(true);
+      sync((FileOutputStream)checksumOut);
     }
   }
 
+  /**
+   * Flush the data stream if it supports it.
+   */
+  public void flushDataOut() throws IOException {
+    flush(dataOut);
+  }
+
+  /**
+   * Flush the checksum stream if it supports it.
+   */
+  public void flushChecksumOut() throws IOException {
+    flush(checksumOut);
+  }
+
+  private void flush(OutputStream dos) throws IOException {
+    long begin = Time.monotonicNow();
+    dos.flush();
+    long duration = Time.monotonicNow() - begin;
+    LOG.trace("ReplicaOutputStreams#flush takes {} ms.", duration);
+    if (duration > slowLogThresholdMs) {
+      LOG.warn("Slow flush took {} ms (threshold={} ms)", duration,
+          slowLogThresholdMs);
+    }
+  }
+
+  private void sync(FileOutputStream fos) throws IOException {
+    long begin = Time.monotonicNow();
+    fos.getChannel().force(true);
+    long duration = Time.monotonicNow() - begin;
+    LOG.trace("ReplicaOutputStreams#sync takes {} ms.", duration);
+    if (duration > slowLogThresholdMs) {
+      LOG.warn("Slow fsync took {} ms (threshold={} ms)", duration,
+          slowLogThresholdMs);
+    }
+  }
+
+  public long writeToDisk(byte[] b, int off, int len) throws IOException {
+    long begin = Time.monotonicNow();
+    dataOut.write(b, off, len);
+    long duration = Time.monotonicNow() - begin;
+    LOG.trace("DatanodeIO#writeToDisk takes {} ms.", duration);
+    if (duration > slowLogThresholdMs) {
+      LOG.warn("Slow BlockReceiver write data to disk cost: {} ms " +
+          "(threshold={} ms)", duration, slowLogThresholdMs);
+    }
+    return duration;
+  }
+
+  public void syncFileRangeIfPossible(long offset, long nbytes,
+      int flags) throws NativeIOException {
+    assert this.outFd != null : "null outFd!";
+    NativeIO.POSIX.syncFileRangeIfPossible(outFd, offset, nbytes, flags);
+  }
+
+  public void dropCacheBehindWrites(String identifier,
+      long offset, long len, int flags) throws NativeIOException {
+    assert this.outFd != null : "null outFd!";
+    NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+        identifier, outFd, offset, len, flags);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 29dbb29..63e82f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -49,11 +49,13 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
+import org.apache.hadoop.hdfs.server.datanode.LocalReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
+
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker;
@@ -145,7 +147,7 @@ class BlockPoolSlice {
     //
     this.tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
     if (tmpDir.exists()) {
-      FileUtil.fullyDelete(tmpDir);
+      DataStorage.fullyDelete(tmpDir);
     }
     this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
     if (!rbwDir.mkdirs()) {  // create rbw directory if not exist
@@ -436,7 +438,7 @@ class BlockPoolSlice {
 
           final File targetMetaFile = new File(targetDir, metaFile.getName());
           try {
-            NativeIO.renameTo(metaFile, targetMetaFile);
+            LocalReplica.rename(metaFile, targetMetaFile);
           } catch (IOException e) {
             LOG.warn("Failed to move meta file from "
                 + metaFile + " to " + targetMetaFile, e);
@@ -446,7 +448,7 @@ class BlockPoolSlice {
 
           final File targetBlockFile = new File(targetDir, blockFile.getName());
           try {
-            NativeIO.renameTo(blockFile, targetBlockFile);
+            LocalReplica.rename(blockFile, targetBlockFile);
           } catch (IOException e) {
             LOG.warn("Failed to move block file from "
                 + blockFile + " to " + targetBlockFile, e);
@@ -688,8 +690,6 @@ class BlockPoolSlice {
    * @return the number of valid bytes
    */
   private long validateIntegrityAndSetLength(File blockFile, long genStamp) {
-    DataInputStream checksumIn = null;
-    InputStream blockIn = null;
     try {
       final File metaFile = FsDatasetUtil.getMetaFile(blockFile, genStamp);
       long blockFileLen = blockFile.length();
@@ -699,57 +699,52 @@ class BlockPoolSlice {
           !metaFile.exists() || metaFileLen < crcHeaderLen) {
         return 0;
       }
-      checksumIn = new DataInputStream(
+      try (DataInputStream checksumIn = new DataInputStream(
           new BufferedInputStream(new FileInputStream(metaFile),
-              ioFileBufferSize));
-
-      // read and handle the common header here. For now just a version
-      final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(
-          checksumIn, metaFile);
-      int bytesPerChecksum = checksum.getBytesPerChecksum();
-      int checksumSize = checksum.getChecksumSize();
-      long numChunks = Math.min(
-          (blockFileLen + bytesPerChecksum - 1)/bytesPerChecksum,
-          (metaFileLen - crcHeaderLen)/checksumSize);
-      if (numChunks == 0) {
-        return 0;
-      }
-      IOUtils.skipFully(checksumIn, (numChunks-1)*checksumSize);
-      blockIn = new FileInputStream(blockFile);
-      long lastChunkStartPos = (numChunks-1)*bytesPerChecksum;
-      IOUtils.skipFully(blockIn, lastChunkStartPos);
-      int lastChunkSize = (int)Math.min(
-          bytesPerChecksum, blockFileLen-lastChunkStartPos);
-      byte[] buf = new byte[lastChunkSize+checksumSize];
-      checksumIn.readFully(buf, lastChunkSize, checksumSize);
-      IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
-
-      checksum.update(buf, 0, lastChunkSize);
-      long validFileLength;
-      if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
-        validFileLength = lastChunkStartPos + lastChunkSize;
-      } else { // last chunck is corrupt
-        validFileLength = lastChunkStartPos;
-      }
-
-      // truncate if extra bytes are present without CRC
-      if (blockFile.length() > validFileLength) {
-        RandomAccessFile blockRAF = new RandomAccessFile(blockFile, "rw");
-        try {
-          // truncate blockFile
-          blockRAF.setLength(validFileLength);
-        } finally {
-          blockRAF.close();
+              ioFileBufferSize))) {
+        // read and handle the common header here. For now just a version
+        final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(
+            checksumIn, metaFile);
+        int bytesPerChecksum = checksum.getBytesPerChecksum();
+        int checksumSize = checksum.getChecksumSize();
+        long numChunks = Math.min(
+            (blockFileLen + bytesPerChecksum - 1) / bytesPerChecksum,
+            (metaFileLen - crcHeaderLen) / checksumSize);
+        if (numChunks == 0) {
+          return 0;
+        }
+        try (InputStream blockIn = new FileInputStream(blockFile);
+             ReplicaInputStreams ris = new ReplicaInputStreams(blockIn,
+                 checksumIn, volume.obtainReference())) {
+          ris.skipChecksumFully((numChunks - 1) * checksumSize);
+          long lastChunkStartPos = (numChunks - 1) * bytesPerChecksum;
+          ris.skipDataFully(lastChunkStartPos);
+          int lastChunkSize = (int) Math.min(
+              bytesPerChecksum, blockFileLen - lastChunkStartPos);
+          byte[] buf = new byte[lastChunkSize + checksumSize];
+          ris.readChecksumFully(buf, lastChunkSize, checksumSize);
+          ris.readDataFully(buf, 0, lastChunkSize);
+          checksum.update(buf, 0, lastChunkSize);
+          long validFileLength;
+          if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
+            validFileLength = lastChunkStartPos + lastChunkSize;
+          } else { // last chunk is corrupt
+            validFileLength = lastChunkStartPos;
+          }
+          // truncate if extra bytes are present without CRC
+          if (blockFile.length() > validFileLength) {
+            try (RandomAccessFile blockRAF =
+                     new RandomAccessFile(blockFile, "rw")) {
+              // truncate blockFile
+              blockRAF.setLength(validFileLength);
+            }
+          }
+          return validFileLength;
         }
       }
-
-      return validFileLength;
     } catch (IOException e) {
       FsDatasetImpl.LOG.warn(e);
       return 0;
-    } finally {
-      IOUtils.closeStream(checksumIn);
-      IOUtils.closeStream(blockIn);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
index b9c731b..97dcf8d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -38,9 +37,9 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
 
 /**
@@ -202,13 +201,13 @@ class FsDatasetAsyncDiskService {
   }
 
   public void submitSyncFileRangeRequest(FsVolumeImpl volume,
-      final FileDescriptor fd, final long offset, final long nbytes,
+      final ReplicaOutputStreams streams, final long offset, final long nbytes,
       final int flags) {
     execute(volume, new Runnable() {
       @Override
       public void run() {
         try {
-          NativeIO.POSIX.syncFileRangeIfPossible(fd, offset, nbytes, flags);
+          streams.syncFileRangeIfPossible(offset, nbytes, flags);
         } catch (NativeIOException e) {
           LOG.warn("sync_file_range error", e);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 954d6ef..6065df2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -21,7 +21,6 @@ import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -2755,9 +2754,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
   @Override
   public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
-      FileDescriptor fd, long offset, long nbytes, int flags) {
+      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
     FsVolumeImpl fsVolumeImpl = this.getVolume(block);
-    asyncDiskService.submitSyncFileRangeRequest(fsVolumeImpl, fd, offset,
+    asyncDiskService.submitSyncFileRangeRequest(fsVolumeImpl, outs, offset,
         nbytes, flags);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index a231e03..08564de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -1067,7 +1067,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
         DataStorage.STORAGE_DIR_LAZY_PERSIST);
     File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
     if (force) {
-      FileUtil.fullyDelete(bpDir);
+      DataStorage.fullyDelete(bpDir);
     } else {
       if (!rbwDir.delete()) {
         throw new IOException("Failed to delete " + rbwDir);
@@ -1081,7 +1081,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
           !FileUtil.fullyDelete(lazypersistDir)))) {
         throw new IOException("Failed to delete " + lazypersistDir);
       }
-      FileUtil.fullyDelete(tmpDir);
+      DataStorage.fullyDelete(tmpDir);
       for (File f : FileUtil.listFiles(bpCurrentDir)) {
         if (!f.delete()) {
           throw new IOException("Failed to delete " + f);
@@ -1437,4 +1437,3 @@ public class FsVolumeImpl implements FsVolumeSpi {
         replicaState);
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
index 20cec6a..e963d41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
@@ -701,7 +701,7 @@ public class TestFileAppend{
       ReplicaBeingWritten rbw =
           (ReplicaBeingWritten)replicaHandler.getReplica();
       ReplicaOutputStreams
-          outputStreams = rbw.createStreams(false, DEFAULT_CHECKSUM);
+          outputStreams = rbw.createStreams(false, DEFAULT_CHECKSUM, 300);
       OutputStream dataOutput = outputStreams.getDataOut();
 
       byte[] appendBytes = new byte[1];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 5d63d07..ae52905 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -261,14 +260,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
     @Override
     synchronized public ReplicaOutputStreams createStreams(boolean isCreate, 
-        DataChecksum requestedChecksum) throws IOException {
+        DataChecksum requestedChecksum, long slowLogThresholdMs)
+        throws IOException {
       if (finalized) {
         throw new IOException("Trying to write to a finalized replica "
             + theBlock);
       } else {
         SimulatedOutputStream crcStream = new SimulatedOutputStream();
         return new ReplicaOutputStreams(oStream, crcStream, requestedChecksum,
-            volume.isTransientStorage());
+            volume.isTransientStorage(), slowLogThresholdMs);
       }
     }
 
@@ -1364,7 +1364,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override
   public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
-      FileDescriptor fd, long offset, long nbytes, int flags) {
+      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 619eda0..8439991 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -673,7 +673,7 @@ public class TestBlockRecovery {
     ReplicaOutputStreams streams = null;
     try {
       streams = replicaInfo.createStreams(true,
-          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
       streams.getChecksumOut().write('a');
       dn.data.initReplicaRecovery(new RecoveringBlock(block, null, RECOVERY_ID+1));
       BlockRecoveryWorker.RecoveryTaskContiguous RecoveryTaskContiguous =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
index 4e724bc7..fa980c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
@@ -83,7 +83,7 @@ public class TestSimulatedFSDataset {
       ReplicaInPipeline bInfo = fsdataset.createRbw(
           StorageType.DEFAULT, b, false).getReplica();
       ReplicaOutputStreams out = bInfo.createStreams(true,
-          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
       try {
         OutputStream dataOut  = out.getDataOut();
         assertEquals(0, fsdataset.getLength(b));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 4166346..2417c9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -318,8 +318,8 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block, FileDescriptor fd, long offset, long nbytes, int flags) {
-
+  public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
+      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df983b52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
index 90c3b8a..6fa2830 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
@@ -58,8 +58,10 @@ public class ExternalReplicaInPipeline implements ReplicaInPipeline {
 
   @Override
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum) throws IOException {
-    return new ReplicaOutputStreams(null, null, requestedChecksum, false);
+      DataChecksum requestedChecksum, long slowLogThresholdMs)
+      throws IOException {
+    return new ReplicaOutputStreams(null, null, requestedChecksum, false,
+        slowLogThresholdMs);
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/50] [abbrv] hadoop git commit: Revert "HDFS-11201. Spelling errors in the logging, help, assertions and exception messages. Contributed by Grant Sohn."

Posted by st...@apache.org.
Revert "HDFS-11201. Spelling errors in the logging, help, assertions and exception messages. Contributed by Grant Sohn."

This reverts commit b9522e86a55564c2ccb5ca3f1ca871965cbe74de.


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

Branch: refs/heads/HADOOP-13345
Commit: 1b5cceaffbdde50a87ede81552dc380832db8e79
Parents: b9522e8
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Mon Dec 5 10:54:43 2016 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Mon Dec 5 10:54:43 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java     | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java | 4 ++--
 .../main/java/org/apache/hadoop/lib/server/ServerException.java  | 2 +-
 .../java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java   | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/DFSUtil.java            | 2 +-
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java  | 2 +-
 .../hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java      | 2 +-
 .../hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java | 2 +-
 .../hadoop/hdfs/server/diskbalancer/command/QueryCommand.java    | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java  | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java  | 2 +-
 .../java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java   | 4 ++--
 .../server/namenode/web/resources/NamenodeWebHdfsMethods.java    | 2 +-
 13 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index aabd6fd..5783f90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1052,7 +1052,7 @@ public class DFSInputStream extends FSInputStream
             reader.getNetworkDistance(), nread);
         if (nread != len) {
           throw new IOException("truncated return from reader.read(): " +
-              "expected " + len + ", got " + nread);
+              "excpected " + len + ", got " + nread);
         }
         DFSClientFaultInjector.get().readFromDatanodeDelay();
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
index db064e4..51ad08f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
@@ -57,11 +57,11 @@ public class LongBitFormat implements Serializable {
   public long combine(long value, long record) {
     if (value < MIN) {
       throw new IllegalArgumentException(
-          "Illegal value: " + NAME + " = " + value + " < MIN = " + MIN);
+          "Illagal value: " + NAME + " = " + value + " < MIN = " + MIN);
     }
     if (value > MAX) {
       throw new IllegalArgumentException(
-          "Illegal value: " + NAME + " = " + value + " > MAX = " + MAX);
+          "Illagal value: " + NAME + " = " + value + " > MAX = " + MAX);
     }
     return (record & ~MASK) | (value << OFFSET);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
index fdca64e..e3759ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
@@ -38,7 +38,7 @@ public class ServerException extends XException {
     S04("Service [{0}] does not implement declared interface [{1}]"),
     S05("[{0}] is not a file"),
     S06("Could not load file [{0}], {1}"),
-    S07("Could not instantiate service class [{0}], {1}"),
+    S07("Could not instanciate service class [{0}], {1}"),
     S08("Could not load service classes, {0}"),
     S09("Could not set service [{0}] programmatically -server shutting down-, {1}"),
     S10("Service [{0}] requires service [{1}]"),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
index 2d8f676..e26fac5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
@@ -130,7 +130,7 @@ class OpenFileCtxCache {
           }
           toEvict = openFileMap.remove(pairs.getKey());
           Preconditions.checkState(toEvict == pairs.getValue(),
-              "The deleted entry is not the same as oldest found.");
+              "The deleted entry is not the same as odlest found.");
         }
       }
       openFileMap.put(h, context);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 4664699..23166e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -1364,7 +1364,7 @@ public class DFSUtil {
         DFSConfigKeys.DFS_HTTP_POLICY_DEFAULT);
     HttpConfig.Policy policy = HttpConfig.Policy.fromString(policyStr);
     if (policy == null) {
-      throw new HadoopIllegalArgumentException("Unrecognized value '"
+      throw new HadoopIllegalArgumentException("Unregonized value '"
           + policyStr + "' for " + DFSConfigKeys.DFS_HTTP_POLICY_KEY);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 05d538a..e60703b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2772,7 +2772,7 @@ public class BlockManager implements BlockStatsMXBean {
       throws IOException {
     for (ReportedBlockInfo rbi : rbis) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Processing previously queued message " + rbi);
+        LOG.debug("Processing previouly queued message " + rbi);
       }
       if (rbi.getReportedState() == null) {
         // This is a DELETE_BLOCK request

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 69dc9f9..8323140 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -836,7 +836,7 @@ class BlockPoolSlice {
     } catch (Exception e) {
       // Any exception we need to revert back to read from disk
       // Log the error and return false
-      LOG.info("Exception occurred while reading the replicas cache file: "
+      LOG.info("Exception occured while reading the replicas cache file: "
           + replicaFile.getPath(), e );
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index d41f9c3..08564de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -1292,7 +1292,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
     try {
       fileNames = IOUtils.listDirectory(dir, BlockDirFilter.INSTANCE);
     } catch (IOException ioe) {
-      LOG.warn("Exception occurred while compiling report: ", ioe);
+      LOG.warn("Exception occured while compiling report: ", ioe);
       // Initiate a check on disk failure.
       dataset.datanode.checkDiskErrorAsync();
       // Ignore this directory and proceed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
index 9e60e48..a8adcbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -97,7 +97,7 @@ public class QueryCommand extends Command {
     String header = "Query Plan queries a given data node about the " +
         "current state of disk balancer execution.\n\n";
 
-    String footer = "\nQuery command retrieves the plan ID and the current " +
+    String footer = "\nQuery command retrievs the plan ID and the current " +
         "running state. ";
 
     HelpFormatter helpFormatter = new HelpFormatter();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
index 41ec8e9..f2a1ee5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
@@ -134,7 +134,7 @@ final class FSDirTruncateOp {
       if (!onBlockBoundary) {
         // Open file for write, but don't log into edits
         long lastBlockDelta = file.computeFileSize() - newLength;
-        assert lastBlockDelta > 0 : "delta is 0 only if on block boundary";
+        assert lastBlockDelta > 0 : "delta is 0 only if on block bounday";
         truncateBlock = prepareFileForTruncate(fsn, iip, clientName,
             clientMachine, lastBlockDelta, null);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 11cdbc6..2990344 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -1082,7 +1082,7 @@ public class FSEditLogLoader {
       boolean removed = FSDirWriteFileOp.unprotectedRemoveBlock(
           fsDir, path, iip, file, oldBlock);
       if (!removed && !(op instanceof UpdateBlocksOp)) {
-        throw new IOException("Trying to delete non-existent block " + oldBlock);
+        throw new IOException("Trying to delete non-existant block " + oldBlock);
       }
     } else if (newBlocks.length > oldBlocks.length) {
       final boolean isStriped = ecPolicy != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index f79130d..d6dd8ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -1025,7 +1025,7 @@ public class NNStorage extends Storage implements Closeable,
     
     if (!blockpoolID.equals("") && !blockpoolID.equals(bpid)) {
       throw new InconsistentFSStateException(storage,
-          "Unexpected blockpoolID " + bpid + " . Expected " + blockpoolID);
+          "Unexepcted blockpoolID " + bpid + " . Expected " + blockpoolID);
     }
     setBlockPoolID(bpid);
   }
@@ -1154,4 +1154,4 @@ public class NNStorage extends Storage implements Closeable,
       }
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b5cceaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 2c31cd9..107d4ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -219,7 +219,7 @@ public class NamenodeWebHdfsMethods {
       final String remoteAddr) throws IOException {
     FSNamesystem fsn = namenode.getNamesystem();
     if (fsn == null) {
-      throw new IOException("Namesystem has not been initialized yet.");
+      throw new IOException("Namesystem has not been intialized yet.");
     }
     final BlockManager bm = fsn.getBlockManager();
     


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/50] [abbrv] hadoop git commit: HADOOP-13835. Move Google Test Framework code from mapreduce to hadoop-common. Contributed by Varun Vasudev.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
index 5f33843..9f12986 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
@@ -92,8 +92,6 @@
         <configuration>
           <excludes>
             <exclude>src/main/native/testData/*</exclude>
-            <!-- gtest has a compatible license -->
-            <exclude>src/main/native/gtest/**/*</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
index ee02062..d9cec1a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt
@@ -21,6 +21,9 @@ cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
 list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../../hadoop-common-project/hadoop-common/)
 include(HadoopCommon)
 
+# Set gtest path
+set(GTEST_SRC_DIR ${CMAKE_SOURCE_DIR}/../../../../hadoop-common-project/hadoop-common/src/main/native/gtest)
+
 # Add extra compiler and linker flags.
 # -Wno-sign-compare
 hadoop_add_compiler_flags("-DNDEBUG -DSIMPLE_MEMCPY -fno-strict-aliasing")
@@ -94,9 +97,10 @@ include_directories(
     ${CMAKE_BINARY_DIR}
     ${JNI_INCLUDE_DIRS}
     ${SNAPPY_INCLUDE_DIR}
+    ${GTEST_SRC_DIR}/include
 )
 # add gtest as system library to suppress gcc warnings
-include_directories(SYSTEM ${SRC}/gtest/include)
+include_directories(SYSTEM ${GTEST_SRC_DIR}/include)
 
 set(CMAKE_MACOSX_RPATH TRUE)
 set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
@@ -155,7 +159,7 @@ hadoop_add_dual_library(nativetask
 
 target_link_libraries(nativetask ${NT_DEPEND_LIBRARY})
 
-add_library(gtest ${SRC}/gtest/gtest-all.cc)
+add_library(gtest ${GTEST_SRC_DIR}/gtest-all.cc)
 set_target_properties(gtest PROPERTIES COMPILE_FLAGS "-w")
 add_executable(nttest
     ${SRC}/test/lib/TestByteArray.cc


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/50] [abbrv] hadoop git commit: YARN-5921. Incorrect synchronization in RMContextImpl#setHAServiceState/getHAServiceState. Contributed by Varun Saxena

Posted by st...@apache.org.
YARN-5921. Incorrect synchronization in RMContextImpl#setHAServiceState/getHAServiceState. Contributed by Varun Saxena


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

Branch: refs/heads/HADOOP-13345
Commit: f3b8ff54ab08545d7093bf8861b44ec9912e8dc3
Parents: dcedb72
Author: Naganarasimha <na...@apache.org>
Authored: Tue Dec 6 06:53:38 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Tue Dec 6 06:53:38 2016 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/RMContextImpl.java | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3b8ff54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index dc8f7d1..3f17ac6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -80,6 +80,8 @@ public class RMContextImpl implements RMContext {
 
   private QueueLimitCalculator queueLimitCalculator;
 
+  private final Object haServiceStateLock = new Object();
+
   /**
    * Default constructor. To be used in conjunction with setter methods for
    * individual fields.
@@ -254,9 +256,9 @@ public class RMContextImpl implements RMContext {
     this.isHAEnabled = isHAEnabled;
   }
 
-  void setHAServiceState(HAServiceState haServiceState) {
-    synchronized (haServiceState) {
-      this.haServiceState = haServiceState;
+  void setHAServiceState(HAServiceState serviceState) {
+    synchronized (haServiceStateLock) {
+      this.haServiceState = serviceState;
     }
   }
 
@@ -352,7 +354,7 @@ public class RMContextImpl implements RMContext {
 
   @Override
   public HAServiceState getHAServiceState() {
-    synchronized (haServiceState) {
+    synchronized (haServiceStateLock) {
       return haServiceState;
     }
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/50] [abbrv] hadoop git commit: HDFS-11172. Support an erasure coding policy using RS 10 + 4. Contributed by Wei Zhou.

Posted by st...@apache.org.
HDFS-11172. Support an erasure coding policy using RS 10 + 4. Contributed by Wei Zhou.


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

Branch: refs/heads/HADOOP-13345
Commit: f53b481ffeec64831c1d9b54a555c7b570d116a3
Parents: 79d90b8
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Dec 6 12:49:38 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Dec 6 12:49:38 2016 -0800

----------------------------------------------------------------------
 .../io/erasurecode/ErasureCodeConstants.java    |  3 ++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  1 +
 .../namenode/ErasureCodingPolicyManager.java    |  5 ++-
 .../src/site/markdown/HDFSErasureCoding.md      |  2 +-
 .../TestDFSRSDefault10x4StripedInputStream.java | 35 +++++++++++++++++++
 ...TestDFSRSDefault10x4StripedOutputStream.java | 36 ++++++++++++++++++++
 ...fault10x4StripedOutputStreamWithFailure.java | 36 ++++++++++++++++++++
 7 files changed, 116 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f53b481f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
index ffa0bce..e168909 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
@@ -41,4 +41,7 @@ public final class ErasureCodeConstants {
 
   public static final ECSchema XOR_2_1_SCHEMA = new ECSchema(
       XOR_CODEC_NAME, 2, 1);
+
+  public static final ECSchema RS_10_4_SCHEMA = new ECSchema(
+      RS_DEFAULT_CODEC_NAME, 10, 4);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f53b481f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index b55b4df..a9f1839 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -148,6 +148,7 @@ public final class HdfsConstants {
   public static final byte RS_3_2_POLICY_ID = 1;
   public static final byte RS_6_3_LEGACY_POLICY_ID = 2;
   public static final byte XOR_2_1_POLICY_ID = 3;
+  public static final byte RS_10_4_POLICY_ID = 4;
 
   /* Hidden constructor */
   protected HdfsConstants() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f53b481f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 8a85d23..a1b2270 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -51,11 +51,14 @@ public final class ErasureCodingPolicyManager {
   private static final ErasureCodingPolicy SYS_POLICY4 =
       new ErasureCodingPolicy(ErasureCodeConstants.XOR_2_1_SCHEMA,
           DEFAULT_CELLSIZE, HdfsConstants.XOR_2_1_POLICY_ID);
+  private static final ErasureCodingPolicy SYS_POLICY5 =
+      new ErasureCodingPolicy(ErasureCodeConstants.RS_10_4_SCHEMA,
+          DEFAULT_CELLSIZE, HdfsConstants.RS_10_4_POLICY_ID);
 
   //We may add more later.
   private static final ErasureCodingPolicy[] SYS_POLICIES =
       new ErasureCodingPolicy[]{SYS_POLICY1, SYS_POLICY2, SYS_POLICY3,
-          SYS_POLICY4};
+          SYS_POLICY4, SYS_POLICY5};
 
   // Supported storage policies for striped EC files
   private static final byte[] SUITABLE_STORAGE_POLICIES_FOR_EC_STRIPED_MODE = new byte[] {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f53b481f/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index dbd1f44..517469d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -72,7 +72,7 @@ Architecture
 
       2. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
 
-    There are three policies currently being supported: RS-DEFAULT-3-2-64k, RS-DEFAULT-6-3-64k and RS-LEGACY-6-3-64k. All with default cell size of 64KB. The system default policy is RS-DEFAULT-6-3-64k which use the default schema RS_6_3_SCHEMA with a cell size of 64KB.
+    There are four policies currently being supported: RS-DEFAULT-3-2-64k, RS-DEFAULT-6-3-64k, RS-DEFAULT-10-4-64k and RS-LEGACY-6-3-64k. All with default cell size of 64KB. The system default policy is RS-DEFAULT-6-3-64k which use the default schema RS_6_3_SCHEMA with a cell size of 64KB.
 
  *  **Intel ISA-L**
     Intel ISA-L stands for Intel Intelligent Storage Acceleration Library. ISA-L is a collection of optimized low-level functions used primarily in storage applications. It includes a fast block Reed-Solomon type erasure codes optimized for Intel AVX and AVX2 instruction sets.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f53b481f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedInputStream.java
new file mode 100644
index 0000000..fc0ee37
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedInputStream.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+
+/**
+ * This tests read operation of DFS striped file with RS-DEFAULT-10-4-64k
+ *  erasure code policy.
+ */
+public class TestDFSRSDefault10x4StripedInputStream extends
+    TestDFSStripedInputStream {
+
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getPolicyByPolicyID(
+        HdfsConstants.RS_10_4_POLICY_ID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f53b481f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStream.java
new file mode 100644
index 0000000..37821c1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStream.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+
+/**
+ * This tests write operation of DFS striped file with RS-DEFAULT-10-4-64k
+ *  erasure code policy.
+ */
+public class TestDFSRSDefault10x4StripedOutputStream
+    extends TestDFSStripedOutputStream {
+
+  @Override
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getPolicyByPolicyID(
+        HdfsConstants.RS_10_4_POLICY_ID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f53b481f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStreamWithFailure.java
new file mode 100644
index 0000000..1b2ec42
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStreamWithFailure.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+
+/**
+ * This tests write operation of DFS striped file with RS-DEFAULT-10-4-64k
+ *  erasure code policy under Datanode failure conditions.
+ */
+public class TestDFSRSDefault10x4StripedOutputStreamWithFailure
+    extends TestDFSStripedOutputStreamWithFailure {
+
+  @Override
+  public ErasureCodingPolicy getEcPolicy() {
+    return ErasureCodingPolicyManager.getPolicyByPolicyID(
+        HdfsConstants.RS_10_4_POLICY_ID);
+  }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/50] [abbrv] hadoop git commit: HADOOP-13835. Move Google Test Framework code from mapreduce to hadoop-common. Contributed by Varun Vasudev.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/hadoop-common-project/hadoop-common/src/main/native/gtest/include/gtest/gtest.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/gtest/include/gtest/gtest.h b/hadoop-common-project/hadoop-common/src/main/native/gtest/include/gtest/gtest.h
new file mode 100644
index 0000000..c04205d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/gtest/include/gtest/gtest.h
@@ -0,0 +1,21192 @@
+// Copyright 2005, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Author: wan@google.com (Zhanyong Wan)
+//
+// The Google C++ Testing Framework (Google Test)
+//
+// This header file defines the public API for Google Test.  It should be
+// included by any test program that uses Google Test.
+//
+// IMPORTANT NOTE: Due to limitation of the C++ language, we have to
+// leave some internal implementation details in this header file.
+// They are clearly marked by comments like this:
+//
+//   // INTERNAL IMPLEMENTATION - DO NOT USE IN A USER PROGRAM.
+//
+// Such code is NOT meant to be used by a user directly, and is subject
+// to CHANGE WITHOUT NOTICE.  Therefore DO NOT DEPEND ON IT in a user
+// program!
+//
+// Acknowledgment: Google Test borrowed the idea of automatic test
+// registration from Barthelemy Dagenais' (barthelemy@prologique.com)
+// easyUnit framework.
+
+#ifndef GTEST_INCLUDE_GTEST_GTEST_H_
+#define GTEST_INCLUDE_GTEST_GTEST_H_
+
+#include <limits>
+#include <ostream>
+#include <vector>
+
+// Copyright 2005, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Authors: wan@google.com (Zhanyong Wan), eefacm@gmail.com (Sean Mcafee)
+//
+// The Google C++ Testing Framework (Google Test)
+//
+// This header file declares functions and macros used internally by
+// Google Test.  They are subject to change without notice.
+
+#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_INTERNAL_H_
+#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_INTERNAL_H_
+
+// Copyright 2005, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Authors: wan@google.com (Zhanyong Wan)
+//
+// Low-level types and utilities for porting Google Test to various
+// platforms.  All macros ending with _ and symbols defined in an
+// internal namespace are subject to change without notice.  Code
+// outside Google Test MUST NOT USE THEM DIRECTLY.  Macros that don't
+// end with _ are part of Google Test's public API and can be used by
+// code outside Google Test.
+//
+// This file is fundamental to Google Test.  All other Google Test source
+// files are expected to #include this.  Therefore, it cannot #include
+// any other Google Test header.
+
+#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_H_
+#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_H_
+
+// Environment-describing macros
+// -----------------------------
+//
+// Google Test can be used in many different environments.  Macros in
+// this section tell Google Test what kind of environment it is being
+// used in, such that Google Test can provide environment-specific
+// features and implementations.
+//
+// Google Test tries to automatically detect the properties of its
+// environment, so users usually don't need to worry about these
+// macros.  However, the automatic detection is not perfect.
+// Sometimes it's necessary for a user to define some of the following
+// macros in the build script to override Google Test's decisions.
+//
+// If the user doesn't define a macro in the list, Google Test will
+// provide a default definition.  After this header is #included, all
+// macros in this list will be defined to either 1 or 0.
+//
+// Notes to maintainers:
+//   - Each macro here is a user-tweakable knob; do not grow the list
+//     lightly.
+//   - Use #if to key off these macros.  Don't use #ifdef or "#if
+//     defined(...)", which will not work as these macros are ALWAYS
+//     defined.
+//
+//   GTEST_HAS_CLONE          - Define it to 1/0 to indicate that clone(2)
+//                              is/isn't available.
+//   GTEST_HAS_EXCEPTIONS     - Define it to 1/0 to indicate that exceptions
+//                              are enabled.
+//   GTEST_HAS_GLOBAL_STRING  - Define it to 1/0 to indicate that ::string
+//                              is/isn't available (some systems define
+//                              ::string, which is different to std::string).
+//   GTEST_HAS_GLOBAL_WSTRING - Define it to 1/0 to indicate that ::string
+//                              is/isn't available (some systems define
+//                              ::wstring, which is different to std::wstring).
+//   GTEST_HAS_POSIX_RE       - Define it to 1/0 to indicate that POSIX regular
+//                              expressions are/aren't available.
+//   GTEST_HAS_PTHREAD        - Define it to 1/0 to indicate that <pthread.h>
+//                              is/isn't available.
+//   GTEST_HAS_RTTI           - Define it to 1/0 to indicate that RTTI is/isn't
+//                              enabled.
+//   GTEST_HAS_STD_WSTRING    - Define it to 1/0 to indicate that
+//                              std::wstring does/doesn't work (Google Test can
+//                              be used where std::wstring is unavailable).
+//   GTEST_HAS_TR1_TUPLE      - Define it to 1/0 to indicate tr1::tuple
+//                              is/isn't available.
+//   GTEST_HAS_SEH            - Define it to 1/0 to indicate whether the
+//                              compiler supports Microsoft's "Structured
+//                              Exception Handling".
+//   GTEST_HAS_STREAM_REDIRECTION
+//                            - Define it to 1/0 to indicate whether the
+//                              platform supports I/O stream redirection using
+//                              dup() and dup2().
+//   GTEST_USE_OWN_TR1_TUPLE  - Define it to 1/0 to indicate whether Google
+//                              Test's own tr1 tuple implementation should be
+//                              used.  Unused when the user sets
+//                              GTEST_HAS_TR1_TUPLE to 0.
+//   GTEST_LANG_CXX11         - Define it to 1/0 to indicate that Google Test
+//                              is building in C++11/C++98 mode.
+//   GTEST_LINKED_AS_SHARED_LIBRARY
+//                            - Define to 1 when compiling tests that use
+//                              Google Test as a shared library (known as
+//                              DLL on Windows).
+//   GTEST_CREATE_SHARED_LIBRARY
+//                            - Define to 1 when compiling Google Test itself
+//                              as a shared library.
+
+// Platform-indicating macros
+// --------------------------
+//
+// Macros indicating the platform on which Google Test is being used
+// (a macro is defined to 1 if compiled on the given platform;
+// otherwise UNDEFINED -- it's never defined to 0.).  Google Test
+// defines these macros automatically.  Code outside Google Test MUST
+// NOT define them.
+//
+//   GTEST_OS_AIX      - IBM AIX
+//   GTEST_OS_CYGWIN   - Cygwin
+//   GTEST_OS_FREEBSD  - FreeBSD
+//   GTEST_OS_HPUX     - HP-UX
+//   GTEST_OS_LINUX    - Linux
+//     GTEST_OS_LINUX_ANDROID - Google Android
+//   GTEST_OS_MAC      - Mac OS X
+//     GTEST_OS_IOS    - iOS
+//   GTEST_OS_NACL     - Google Native Client (NaCl)
+//   GTEST_OS_OPENBSD  - OpenBSD
+//   GTEST_OS_QNX      - QNX
+//   GTEST_OS_SOLARIS  - Sun Solaris
+//   GTEST_OS_SYMBIAN  - Symbian
+//   GTEST_OS_WINDOWS  - Windows (Desktop, MinGW, or Mobile)
+//     GTEST_OS_WINDOWS_DESKTOP  - Windows Desktop
+//     GTEST_OS_WINDOWS_MINGW    - MinGW
+//     GTEST_OS_WINDOWS_MOBILE   - Windows Mobile
+//     GTEST_OS_WINDOWS_PHONE    - Windows Phone
+//     GTEST_OS_WINDOWS_RT       - Windows Store App/WinRT
+//   GTEST_OS_ZOS      - z/OS
+//
+// Among the platforms, Cygwin, Linux, Max OS X, and Windows have the
+// most stable support.  Since core members of the Google Test project
+// don't have access to other platforms, support for them may be less
+// stable.  If you notice any problems on your platform, please notify
+// googletestframework@googlegroups.com (patches for fixing them are
+// even more welcome!).
+//
+// It is possible that none of the GTEST_OS_* macros are defined.
+
+// Feature-indicating macros
+// -------------------------
+//
+// Macros indicating which Google Test features are available (a macro
+// is defined to 1 if the corresponding feature is supported;
+// otherwise UNDEFINED -- it's never defined to 0.).  Google Test
+// defines these macros automatically.  Code outside Google Test MUST
+// NOT define them.
+//
+// These macros are public so that portable tests can be written.
+// Such tests typically surround code using a feature with an #if
+// which controls that code.  For example:
+//
+// #if GTEST_HAS_DEATH_TEST
+//   EXPECT_DEATH(DoSomethingDeadly());
+// #endif
+//
+//   GTEST_HAS_COMBINE      - the Combine() function (for value-parameterized
+//                            tests)
+//   GTEST_HAS_DEATH_TEST   - death tests
+//   GTEST_HAS_PARAM_TEST   - value-parameterized tests
+//   GTEST_HAS_TYPED_TEST   - typed tests
+//   GTEST_HAS_TYPED_TEST_P - type-parameterized tests
+//   GTEST_IS_THREADSAFE    - Google Test is thread-safe.
+//   GTEST_USES_POSIX_RE    - enhanced POSIX regex is used. Do not confuse with
+//                            GTEST_HAS_POSIX_RE (see above) which users can
+//                            define themselves.
+//   GTEST_USES_SIMPLE_RE   - our own simple regex is used;
+//                            the above two are mutually exclusive.
+//   GTEST_CAN_COMPARE_NULL - accepts untyped NULL in EXPECT_EQ().
+
+// Misc public macros
+// ------------------
+//
+//   GTEST_FLAG(flag_name)  - references the variable corresponding to
+//                            the given Google Test flag.
+
+// Internal utilities
+// ------------------
+//
+// The following macros and utilities are for Google Test's INTERNAL
+// use only.  Code outside Google Test MUST NOT USE THEM DIRECTLY.
+//
+// Macros for basic C++ coding:
+//   GTEST_AMBIGUOUS_ELSE_BLOCKER_ - for disabling a gcc warning.
+//   GTEST_ATTRIBUTE_UNUSED_  - declares that a class' instances or a
+//                              variable don't have to be used.
+//   GTEST_DISALLOW_ASSIGN_   - disables operator=.
+//   GTEST_DISALLOW_COPY_AND_ASSIGN_ - disables copy ctor and operator=.
+//   GTEST_MUST_USE_RESULT_   - declares that a function's result must be used.
+//   GTEST_INTENTIONAL_CONST_COND_PUSH_ - start code section where MSVC C4127 is
+//                                        suppressed (constant conditional).
+//   GTEST_INTENTIONAL_CONST_COND_POP_  - finish code section where MSVC C4127
+//                                        is suppressed.
+//
+// C++11 feature wrappers:
+//
+//   testing::internal::move  - portability wrapper for std::move.
+//
+// Synchronization:
+//   Mutex, MutexLock, ThreadLocal, GetThreadCount()
+//                            - synchronization primitives.
+//
+// Template meta programming:
+//   is_pointer     - as in TR1; needed on Symbian and IBM XL C/C++ only.
+//   IteratorTraits - partial implementation of std::iterator_traits, which
+//                    is not available in libCstd when compiled with Sun C++.
+//
+// Smart pointers:
+//   scoped_ptr     - as in TR2.
+//
+// Regular expressions:
+//   RE             - a simple regular expression class using the POSIX
+//                    Extended Regular Expression syntax on UNIX-like
+//                    platforms, or a reduced regular exception syntax on
+//                    other platforms, including Windows.
+//
+// Logging:
+//   GTEST_LOG_()   - logs messages at the specified severity level.
+//   LogToStderr()  - directs all log messages to stderr.
+//   FlushInfoLog() - flushes informational log messages.
+//
+// Stdout and stderr capturing:
+//   CaptureStdout()     - starts capturing stdout.
+//   GetCapturedStdout() - stops capturing stdout and returns the captured
+//                         string.
+//   CaptureStderr()     - starts capturing stderr.
+//   GetCapturedStderr() - stops capturing stderr and returns the captured
+//                         string.
+//
+// Integer types:
+//   TypeWithSize   - maps an integer to a int type.
+//   Int32, UInt32, Int64, UInt64, TimeInMillis
+//                  - integers of known sizes.
+//   BiggestInt     - the biggest signed integer type.
+//
+// Command-line utilities:
+//   GTEST_DECLARE_*()  - declares a flag.
+//   GTEST_DEFINE_*()   - defines a flag.
+//   GetInjectableArgvs() - returns the command line as a vector of strings.
+//
+// Environment variable utilities:
+//   GetEnv()             - gets the value of an environment variable.
+//   BoolFromGTestEnv()   - parses a bool environment variable.
+//   Int32FromGTestEnv()  - parses an Int32 environment variable.
+//   StringFromGTestEnv() - parses a string environment variable.
+
+#include <ctype.h>   // for isspace, etc
+#include <stddef.h>  // for ptrdiff_t
+#include <stdlib.h>
+#include <stdio.h>
+#include <string.h>
+#ifndef _WIN32_WCE
+# include <sys/types.h>
+# include <sys/stat.h>
+#endif  // !_WIN32_WCE
+
+#if defined __APPLE__
+# include <AvailabilityMacros.h>
+# include <TargetConditionals.h>
+#endif
+
+#include <algorithm>  // NOLINT
+#include <iostream>  // NOLINT
+#include <sstream>  // NOLINT
+#include <string>  // NOLINT
+#include <utility>
+#include <vector>  // NOLINT
+
+// Copyright 2015, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// The Google C++ Testing Framework (Google Test)
+//
+// This header file defines the GTEST_OS_* macro.
+// It is separate from gtest-port.h so that custom/gtest-port.h can include it.
+
+#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_ARCH_H_
+#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_ARCH_H_
+
+// Determines the platform on which Google Test is compiled.
+#ifdef __CYGWIN__
+# define GTEST_OS_CYGWIN 1
+#elif defined __SYMBIAN32__
+# define GTEST_OS_SYMBIAN 1
+#elif defined _WIN32
+# define GTEST_OS_WINDOWS 1
+# ifdef _WIN32_WCE
+#  define GTEST_OS_WINDOWS_MOBILE 1
+# elif defined(__MINGW__) || defined(__MINGW32__)
+#  define GTEST_OS_WINDOWS_MINGW 1
+# elif defined(WINAPI_FAMILY)
+#  include <winapifamily.h>
+#  if WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_DESKTOP)
+#   define GTEST_OS_WINDOWS_DESKTOP 1
+#  elif WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_PHONE_APP)
+#   define GTEST_OS_WINDOWS_PHONE 1
+#  elif WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_APP)
+#   define GTEST_OS_WINDOWS_RT 1
+#  else
+    // WINAPI_FAMILY defined but no known partition matched.
+    // Default to desktop.
+#   define GTEST_OS_WINDOWS_DESKTOP 1
+#  endif
+# else
+#  define GTEST_OS_WINDOWS_DESKTOP 1
+# endif  // _WIN32_WCE
+#elif defined __APPLE__
+# define GTEST_OS_MAC 1
+# if TARGET_OS_IPHONE
+#  define GTEST_OS_IOS 1
+# endif
+#elif defined __FreeBSD__
+# define GTEST_OS_FREEBSD 1
+#elif defined __linux__
+# define GTEST_OS_LINUX 1
+# if defined __ANDROID__
+#  define GTEST_OS_LINUX_ANDROID 1
+# endif
+#elif defined __MVS__
+# define GTEST_OS_ZOS 1
+#elif defined(__sun) && defined(__SVR4)
+# define GTEST_OS_SOLARIS 1
+#elif defined(_AIX)
+# define GTEST_OS_AIX 1
+#elif defined(__hpux)
+# define GTEST_OS_HPUX 1
+#elif defined __native_client__
+# define GTEST_OS_NACL 1
+#elif defined __OpenBSD__
+# define GTEST_OS_OPENBSD 1
+#elif defined __QNX__
+# define GTEST_OS_QNX 1
+#endif  // __CYGWIN__
+
+#endif  // GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_ARCH_H_
+// Copyright 2015, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Injection point for custom user configurations.
+// The following macros can be defined:
+//
+//   Flag related macros:
+//     GTEST_FLAG(flag_name)
+//     GTEST_USE_OWN_FLAGFILE_FLAG_  - Define to 0 when the system provides its
+//                                     own flagfile flag parsing.
+//     GTEST_DECLARE_bool_(name)
+//     GTEST_DECLARE_int32_(name)
+//     GTEST_DECLARE_string_(name)
+//     GTEST_DEFINE_bool_(name, default_val, doc)
+//     GTEST_DEFINE_int32_(name, default_val, doc)
+//     GTEST_DEFINE_string_(name, default_val, doc)
+//
+//   Test filtering:
+//     GTEST_TEST_FILTER_ENV_VAR_ - The name of an environment variable that
+//                                  will be used if --GTEST_FLAG(test_filter)
+//                                  is not provided.
+//
+//   Logging:
+//     GTEST_LOG_(severity)
+//     GTEST_CHECK_(condition)
+//     Functions LogToStderr() and FlushInfoLog() have to be provided too.
+//
+//   Threading:
+//     GTEST_HAS_NOTIFICATION_ - Enabled if Notification is already provided.
+//     GTEST_HAS_MUTEX_AND_THREAD_LOCAL_ - Enabled if Mutex and ThreadLocal are
+//                                         already provided.
+//     Must also provide GTEST_DECLARE_STATIC_MUTEX_(mutex) and
+//     GTEST_DEFINE_STATIC_MUTEX_(mutex)
+//
+//     GTEST_EXCLUSIVE_LOCK_REQUIRED_(locks)
+//     GTEST_LOCK_EXCLUDED_(locks)
+//
+// ** Custom implementation starts here **
+
+#ifndef GTEST_INCLUDE_GTEST_INTERNAL_CUSTOM_GTEST_PORT_H_
+#define GTEST_INCLUDE_GTEST_INTERNAL_CUSTOM_GTEST_PORT_H_
+
+#endif  // GTEST_INCLUDE_GTEST_INTERNAL_CUSTOM_GTEST_PORT_H_
+
+#if !defined(GTEST_DEV_EMAIL_)
+# define GTEST_DEV_EMAIL_ "googletestframework@@googlegroups.com"
+# define GTEST_FLAG_PREFIX_ "gtest_"
+# define GTEST_FLAG_PREFIX_DASH_ "gtest-"
+# define GTEST_FLAG_PREFIX_UPPER_ "GTEST_"
+# define GTEST_NAME_ "Google Test"
+# define GTEST_PROJECT_URL_ "https://github.com/google/googletest/"
+#endif  // !defined(GTEST_DEV_EMAIL_)
+
+#if !defined(GTEST_INIT_GOOGLE_TEST_NAME_)
+# define GTEST_INIT_GOOGLE_TEST_NAME_ "testing::InitGoogleTest"
+#endif  // !defined(GTEST_INIT_GOOGLE_TEST_NAME_)
+
+// Determines the version of gcc that is used to compile this.
+#ifdef __GNUC__
+// 40302 means version 4.3.2.
+# define GTEST_GCC_VER_ \
+    (__GNUC__*10000 + __GNUC_MINOR__*100 + __GNUC_PATCHLEVEL__)
+#endif  // __GNUC__
+
+// Macros for disabling Microsoft Visual C++ warnings.
+//
+//   GTEST_DISABLE_MSC_WARNINGS_PUSH_(4800 4385)
+//   /* code that triggers warnings C4800 and C4385 */
+//   GTEST_DISABLE_MSC_WARNINGS_POP_()
+#if _MSC_VER >= 1500
+# define GTEST_DISABLE_MSC_WARNINGS_PUSH_(warnings) \
+    __pragma(warning(push))                        \
+    __pragma(warning(disable: warnings))
+# define GTEST_DISABLE_MSC_WARNINGS_POP_()          \
+    __pragma(warning(pop))
+#else
+// Older versions of MSVC don't have __pragma.
+# define GTEST_DISABLE_MSC_WARNINGS_PUSH_(warnings)
+# define GTEST_DISABLE_MSC_WARNINGS_POP_()
+#endif
+
+#ifndef GTEST_LANG_CXX11
+// gcc and clang define __GXX_EXPERIMENTAL_CXX0X__ when
+// -std={c,gnu}++{0x,11} is passed.  The C++11 standard specifies a
+// value for __cplusplus, and recent versions of clang, gcc, and
+// probably other compilers set that too in C++11 mode.
+# if __GXX_EXPERIMENTAL_CXX0X__ || __cplusplus >= 201103L
+// Compiling in at least C++11 mode.
+#  define GTEST_LANG_CXX11 1
+# else
+#  define GTEST_LANG_CXX11 0
+# endif
+#endif
+
+// Distinct from C++11 language support, some environments don't provide
+// proper C++11 library support. Notably, it's possible to build in
+// C++11 mode when targeting Mac OS X 10.6, which has an old libstdc++
+// with no C++11 support.
+//
+// libstdc++ has sufficient C++11 support as of GCC 4.6.0, __GLIBCXX__
+// 20110325, but maintenance releases in the 4.4 and 4.5 series followed
+// this date, so check for those versions by their date stamps.
+// https://gcc.gnu.org/onlinedocs/libstdc++/manual/abi.html#abi.versioning
+#if GTEST_LANG_CXX11 && \
+    (!defined(__GLIBCXX__) || ( \
+        __GLIBCXX__ >= 20110325ul &&  /* GCC >= 4.6.0 */ \
+        /* Blacklist of patch releases of older branches: */ \
+        __GLIBCXX__ != 20110416ul &&  /* GCC 4.4.6 */ \
+        __GLIBCXX__ != 20120313ul &&  /* GCC 4.4.7 */ \
+        __GLIBCXX__ != 20110428ul &&  /* GCC 4.5.3 */ \
+        __GLIBCXX__ != 20120702ul))   /* GCC 4.5.4 */
+# define GTEST_STDLIB_CXX11 1
+#endif
+
+// Only use C++11 library features if the library provides them.
+#if GTEST_STDLIB_CXX11
+# define GTEST_HAS_STD_BEGIN_AND_END_ 1
+# define GTEST_HAS_STD_FORWARD_LIST_ 1
+# define GTEST_HAS_STD_FUNCTION_ 1
+# define GTEST_HAS_STD_INITIALIZER_LIST_ 1
+# define GTEST_HAS_STD_MOVE_ 1
+# define GTEST_HAS_STD_SHARED_PTR_ 1
+# define GTEST_HAS_STD_TYPE_TRAITS_ 1
+# define GTEST_HAS_STD_UNIQUE_PTR_ 1
+#endif
+
+// C++11 specifies that <tuple> provides std::tuple.
+// Some platforms still might not have it, however.
+#if GTEST_LANG_CXX11
+# define GTEST_HAS_STD_TUPLE_ 1
+# if defined(__clang__)
+// Inspired by http://clang.llvm.org/docs/LanguageExtensions.html#__has_include
+#  if defined(__has_include) && !__has_include(<tuple>)
+#   undef GTEST_HAS_STD_TUPLE_
+#  endif
+# elif defined(_MSC_VER)
+// Inspired by boost/config/stdlib/dinkumware.hpp
+#  if defined(_CPPLIB_VER) && _CPPLIB_VER < 520
+#   undef GTEST_HAS_STD_TUPLE_
+#  endif
+# elif defined(__GLIBCXX__)
+// Inspired by boost/config/stdlib/libstdcpp3.hpp,
+// http://gcc.gnu.org/gcc-4.2/changes.html and
+// http://gcc.gnu.org/onlinedocs/libstdc++/manual/bk01pt01ch01.html#manual.intro.status.standard.200x
+#  if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 2)
+#   undef GTEST_HAS_STD_TUPLE_
+#  endif
+# endif
+#endif
+
+// Brings in definitions for functions used in the testing::internal::posix
+// namespace (read, write, close, chdir, isatty, stat). We do not currently
+// use them on Windows Mobile.
+#if GTEST_OS_WINDOWS
+# if !GTEST_OS_WINDOWS_MOBILE
+#  include <direct.h>
+#  include <io.h>
+# endif
+// In order to avoid having to include <windows.h>, use forward declaration
+// assuming CRITICAL_SECTION is a typedef of _RTL_CRITICAL_SECTION.
+// This assumption is verified by
+// WindowsTypesTest.CRITICAL_SECTIONIs_RTL_CRITICAL_SECTION.
+struct _RTL_CRITICAL_SECTION;
+#else
+// This assumes that non-Windows OSes provide unistd.h. For OSes where this
+// is not the case, we need to include headers that provide the functions
+// mentioned above.
+# include <unistd.h>
+# include <strings.h>
+#endif  // GTEST_OS_WINDOWS
+
+#if GTEST_OS_LINUX_ANDROID
+// Used to define __ANDROID_API__ matching the target NDK API level.
+#  include <android/api-level.h>  // NOLINT
+#endif
+
+// Defines this to true iff Google Test can use POSIX regular expressions.
+#ifndef GTEST_HAS_POSIX_RE
+# if GTEST_OS_LINUX_ANDROID
+// On Android, <regex.h> is only available starting with Gingerbread.
+#  define GTEST_HAS_POSIX_RE (__ANDROID_API__ >= 9)
+# else
+#  define GTEST_HAS_POSIX_RE (!GTEST_OS_WINDOWS)
+# endif
+#endif
+
+#if GTEST_USES_PCRE
+// The appropriate headers have already been included.
+
+#elif GTEST_HAS_POSIX_RE
+
+// On some platforms, <regex.h> needs someone to define size_t, and
+// won't compile otherwise.  We can #include it here as we already
+// included <stdlib.h>, which is guaranteed to define size_t through
+// <stddef.h>.
+# include <regex.h>  // NOLINT
+
+# define GTEST_USES_POSIX_RE 1
+
+#elif GTEST_OS_WINDOWS
+
+// <regex.h> is not available on Windows.  Use our own simple regex
+// implementation instead.
+# define GTEST_USES_SIMPLE_RE 1
+
+#else
+
+// <regex.h> may not be available on this platform.  Use our own
+// simple regex implementation instead.
+# define GTEST_USES_SIMPLE_RE 1
+
+#endif  // GTEST_USES_PCRE
+
+#ifndef GTEST_HAS_EXCEPTIONS
+// The user didn't tell us whether exceptions are enabled, so we need
+// to figure it out.
+# if defined(_MSC_VER) || defined(__BORLANDC__)
+// MSVC's and C++Builder's implementations of the STL use the _HAS_EXCEPTIONS
+// macro to enable exceptions, so we'll do the same.
+// Assumes that exceptions are enabled by default.
+#  ifndef _HAS_EXCEPTIONS
+#   define _HAS_EXCEPTIONS 1
+#  endif  // _HAS_EXCEPTIONS
+#  define GTEST_HAS_EXCEPTIONS _HAS_EXCEPTIONS
+# elif defined(__clang__)
+// clang defines __EXCEPTIONS iff exceptions are enabled before clang 220714,
+// but iff cleanups are enabled after that. In Obj-C++ files, there can be
+// cleanups for ObjC exceptions which also need cleanups, even if C++ exceptions
+// are disabled. clang has __has_feature(cxx_exceptions) which checks for C++
+// exceptions starting at clang r206352, but which checked for cleanups prior to
+// that. To reliably check for C++ exception availability with clang, check for
+// __EXCEPTIONS && __has_feature(cxx_exceptions).
+#  define GTEST_HAS_EXCEPTIONS (__EXCEPTIONS && __has_feature(cxx_exceptions))
+# elif defined(__GNUC__) && __EXCEPTIONS
+// gcc defines __EXCEPTIONS to 1 iff exceptions are enabled.
+#  define GTEST_HAS_EXCEPTIONS 1
+# elif defined(__SUNPRO_CC)
+// Sun Pro CC supports exceptions.  However, there is no compile-time way of
+// detecting whether they are enabled or not.  Therefore, we assume that
+// they are enabled unless the user tells us otherwise.
+#  define GTEST_HAS_EXCEPTIONS 1
+# elif defined(__IBMCPP__) && __EXCEPTIONS
+// xlC defines __EXCEPTIONS to 1 iff exceptions are enabled.
+#  define GTEST_HAS_EXCEPTIONS 1
+# elif defined(__HP_aCC)
+// Exception handling is in effect by default in HP aCC compiler. It has to
+// be turned of by +noeh compiler option if desired.
+#  define GTEST_HAS_EXCEPTIONS 1
+# else
+// For other compilers, we assume exceptions are disabled to be
+// conservative.
+#  define GTEST_HAS_EXCEPTIONS 0
+# endif  // defined(_MSC_VER) || defined(__BORLANDC__)
+#endif  // GTEST_HAS_EXCEPTIONS
+
+#if !defined(GTEST_HAS_STD_STRING)
+// Even though we don't use this macro any longer, we keep it in case
+// some clients still depend on it.
+# define GTEST_HAS_STD_STRING 1
+#elif !GTEST_HAS_STD_STRING
+// The user told us that ::std::string isn't available.
+# error "Google Test cannot be used where ::std::string isn't available."
+#endif  // !defined(GTEST_HAS_STD_STRING)
+
+#ifndef GTEST_HAS_GLOBAL_STRING
+// The user didn't tell us whether ::string is available, so we need
+// to figure it out.
+
+# define GTEST_HAS_GLOBAL_STRING 0
+
+#endif  // GTEST_HAS_GLOBAL_STRING
+
+#ifndef GTEST_HAS_STD_WSTRING
+// The user didn't tell us whether ::std::wstring is available, so we need
+// to figure it out.
+// TODO(wan@google.com): uses autoconf to detect whether ::std::wstring
+//   is available.
+
+// Cygwin 1.7 and below doesn't support ::std::wstring.
+// Solaris' libc++ doesn't support it either.  Android has
+// no support for it at least as recent as Froyo (2.2).
+# define GTEST_HAS_STD_WSTRING \
+    (!(GTEST_OS_LINUX_ANDROID || GTEST_OS_CYGWIN || GTEST_OS_SOLARIS))
+
+#endif  // GTEST_HAS_STD_WSTRING
+
+#ifndef GTEST_HAS_GLOBAL_WSTRING
+// The user didn't tell us whether ::wstring is available, so we need
+// to figure it out.
+# define GTEST_HAS_GLOBAL_WSTRING \
+    (GTEST_HAS_STD_WSTRING && GTEST_HAS_GLOBAL_STRING)
+#endif  // GTEST_HAS_GLOBAL_WSTRING
+
+// Determines whether RTTI is available.
+#ifndef GTEST_HAS_RTTI
+// The user didn't tell us whether RTTI is enabled, so we need to
+// figure it out.
+
+# ifdef _MSC_VER
+
+#  ifdef _CPPRTTI  // MSVC defines this macro iff RTTI is enabled.
+#   define GTEST_HAS_RTTI 1
+#  else
+#   define GTEST_HAS_RTTI 0
+#  endif
+
+// Starting with version 4.3.2, gcc defines __GXX_RTTI iff RTTI is enabled.
+# elif defined(__GNUC__) && (GTEST_GCC_VER_ >= 40302)
+
+#  ifdef __GXX_RTTI
+// When building against STLport with the Android NDK and with
+// -frtti -fno-exceptions, the build fails at link time with undefined
+// references to __cxa_bad_typeid. Note sure if STL or toolchain bug,
+// so disable RTTI when detected.
+#   if GTEST_OS_LINUX_ANDROID && defined(_STLPORT_MAJOR) && \
+       !defined(__EXCEPTIONS)
+#    define GTEST_HAS_RTTI 0
+#   else
+#    define GTEST_HAS_RTTI 1
+#   endif  // GTEST_OS_LINUX_ANDROID && __STLPORT_MAJOR && !__EXCEPTIONS
+#  else
+#   define GTEST_HAS_RTTI 0
+#  endif  // __GXX_RTTI
+
+// Clang defines __GXX_RTTI starting with version 3.0, but its manual recommends
+// using has_feature instead. has_feature(cxx_rtti) is supported since 2.7, the
+// first version with C++ support.
+# elif defined(__clang__)
+
+#  define GTEST_HAS_RTTI __has_feature(cxx_rtti)
+
+// Starting with version 9.0 IBM Visual Age defines __RTTI_ALL__ to 1 if
+// both the typeid and dynamic_cast features are present.
+# elif defined(__IBMCPP__) && (__IBMCPP__ >= 900)
+
+#  ifdef __RTTI_ALL__
+#   define GTEST_HAS_RTTI 1
+#  else
+#   define GTEST_HAS_RTTI 0
+#  endif
+
+# else
+
+// For all other compilers, we assume RTTI is enabled.
+#  define GTEST_HAS_RTTI 1
+
+# endif  // _MSC_VER
+
+#endif  // GTEST_HAS_RTTI
+
+// It's this header's responsibility to #include <typeinfo> when RTTI
+// is enabled.
+#if GTEST_HAS_RTTI
+# include <typeinfo>
+#endif
+
+// Determines whether Google Test can use the pthreads library.
+#ifndef GTEST_HAS_PTHREAD
+// The user didn't tell us explicitly, so we make reasonable assumptions about
+// which platforms have pthreads support.
+//
+// To disable threading support in Google Test, add -DGTEST_HAS_PTHREAD=0
+// to your compiler flags.
+# define GTEST_HAS_PTHREAD (GTEST_OS_LINUX || GTEST_OS_MAC || GTEST_OS_HPUX \
+    || GTEST_OS_QNX || GTEST_OS_FREEBSD || GTEST_OS_NACL)
+#endif  // GTEST_HAS_PTHREAD
+
+#if GTEST_HAS_PTHREAD
+// gtest-port.h guarantees to #include <pthread.h> when GTEST_HAS_PTHREAD is
+// true.
+# include <pthread.h>  // NOLINT
+
+// For timespec and nanosleep, used below.
+# include <time.h>  // NOLINT
+#endif
+
+// Determines if hash_map/hash_set are available.
+// Only used for testing against those containers.
+#if !defined(GTEST_HAS_HASH_MAP_)
+# if _MSC_VER
+#  define GTEST_HAS_HASH_MAP_ 1  // Indicates that hash_map is available.
+#  define GTEST_HAS_HASH_SET_ 1  // Indicates that hash_set is available.
+# endif  // _MSC_VER
+#endif  // !defined(GTEST_HAS_HASH_MAP_)
+
+// Determines whether Google Test can use tr1/tuple.  You can define
+// this macro to 0 to prevent Google Test from using tuple (any
+// feature depending on tuple with be disabled in this mode).
+#ifndef GTEST_HAS_TR1_TUPLE
+# if GTEST_OS_LINUX_ANDROID && defined(_STLPORT_MAJOR)
+// STLport, provided with the Android NDK, has neither <tr1/tuple> or <tuple>.
+#  define GTEST_HAS_TR1_TUPLE 0
+# else
+// The user didn't tell us not to do it, so we assume it's OK.
+#  define GTEST_HAS_TR1_TUPLE 1
+# endif
+#endif  // GTEST_HAS_TR1_TUPLE
+
+// Determines whether Google Test's own tr1 tuple implementation
+// should be used.
+#ifndef GTEST_USE_OWN_TR1_TUPLE
+// The user didn't tell us, so we need to figure it out.
+
+// We use our own TR1 tuple if we aren't sure the user has an
+// implementation of it already.  At this time, libstdc++ 4.0.0+ and
+// MSVC 2010 are the only mainstream standard libraries that come
+// with a TR1 tuple implementation.  NVIDIA's CUDA NVCC compiler
+// pretends to be GCC by defining __GNUC__ and friends, but cannot
+// compile GCC's tuple implementation.  MSVC 2008 (9.0) provides TR1
+// tuple in a 323 MB Feature Pack download, which we cannot assume the
+// user has.  QNX's QCC compiler is a modified GCC but it doesn't
+// support TR1 tuple.  libc++ only provides std::tuple, in C++11 mode,
+// and it can be used with some compilers that define __GNUC__.
+# if (defined(__GNUC__) && !defined(__CUDACC__) && (GTEST_GCC_VER_ >= 40000) \
+      && !GTEST_OS_QNX && !defined(_LIBCPP_VERSION)) || _MSC_VER >= 1600
+#  define GTEST_ENV_HAS_TR1_TUPLE_ 1
+# endif
+
+// C++11 specifies that <tuple> provides std::tuple. Use that if gtest is used
+// in C++11 mode and libstdc++ isn't very old (binaries targeting OS X 10.6
+// can build with clang but need to use gcc4.2's libstdc++).
+# if GTEST_LANG_CXX11 && (!defined(__GLIBCXX__) || __GLIBCXX__ > 20110325)
+#  define GTEST_ENV_HAS_STD_TUPLE_ 1
+# endif
+
+# if GTEST_ENV_HAS_TR1_TUPLE_ || GTEST_ENV_HAS_STD_TUPLE_
+#  define GTEST_USE_OWN_TR1_TUPLE 0
+# else
+#  define GTEST_USE_OWN_TR1_TUPLE 1
+# endif
+
+#endif  // GTEST_USE_OWN_TR1_TUPLE
+
+// To avoid conditional compilation everywhere, we make it
+// gtest-port.h's responsibility to #include the header implementing
+// tuple.
+#if GTEST_HAS_STD_TUPLE_
+# include <tuple>  // IWYU pragma: export
+# define GTEST_TUPLE_NAMESPACE_ ::std
+#endif  // GTEST_HAS_STD_TUPLE_
+
+// We include tr1::tuple even if std::tuple is available to define printers for
+// them.
+#if GTEST_HAS_TR1_TUPLE
+# ifndef GTEST_TUPLE_NAMESPACE_
+#  define GTEST_TUPLE_NAMESPACE_ ::std::tr1
+# endif  // GTEST_TUPLE_NAMESPACE_
+
+# if GTEST_USE_OWN_TR1_TUPLE
+// This file was GENERATED by command:
+//     pump.py gtest-tuple.h.pump
+// DO NOT EDIT BY HAND!!!
+
+// Copyright 2009 Google Inc.
+// All Rights Reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Author: wan@google.com (Zhanyong Wan)
+
+// Implements a subset of TR1 tuple needed by Google Test and Google Mock.
+
+#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_TUPLE_H_
+#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_TUPLE_H_
+
+#include <utility>  // For ::std::pair.
+
+// The compiler used in Symbian has a bug that prevents us from declaring the
+// tuple template as a friend (it complains that tuple is redefined).  This
+// hack bypasses the bug by declaring the members that should otherwise be
+// private as public.
+// Sun Studio versions < 12 also have the above bug.
+#if defined(__SYMBIAN32__) || (defined(__SUNPRO_CC) && __SUNPRO_CC < 0x590)
+# define GTEST_DECLARE_TUPLE_AS_FRIEND_ public:
+#else
+# define GTEST_DECLARE_TUPLE_AS_FRIEND_ \
+    template <GTEST_10_TYPENAMES_(U)> friend class tuple; \
+   private:
+#endif
+
+// Visual Studio 2010, 2012, and 2013 define symbols in std::tr1 that conflict
+// with our own definitions. Therefore using our own tuple does not work on
+// those compilers.
+#if defined(_MSC_VER) && _MSC_VER >= 1600  /* 1600 is Visual Studio 2010 */
+# error "gtest's tuple doesn't compile on Visual Studio 2010 or later. \
+GTEST_USE_OWN_TR1_TUPLE must be set to 0 on those compilers."
+#endif
+
+// GTEST_n_TUPLE_(T) is the type of an n-tuple.
+#define GTEST_0_TUPLE_(T) tuple<>
+#define GTEST_1_TUPLE_(T) tuple<T##0, void, void, void, void, void, void, \
+    void, void, void>
+#define GTEST_2_TUPLE_(T) tuple<T##0, T##1, void, void, void, void, void, \
+    void, void, void>
+#define GTEST_3_TUPLE_(T) tuple<T##0, T##1, T##2, void, void, void, void, \
+    void, void, void>
+#define GTEST_4_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, void, void, void, \
+    void, void, void>
+#define GTEST_5_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, void, void, \
+    void, void, void>
+#define GTEST_6_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, void, \
+    void, void, void>
+#define GTEST_7_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
+    void, void, void>
+#define GTEST_8_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
+    T##7, void, void>
+#define GTEST_9_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
+    T##7, T##8, void>
+#define GTEST_10_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
+    T##7, T##8, T##9>
+
+// GTEST_n_TYPENAMES_(T) declares a list of n typenames.
+#define GTEST_0_TYPENAMES_(T)
+#define GTEST_1_TYPENAMES_(T) typename T##0
+#define GTEST_2_TYPENAMES_(T) typename T##0, typename T##1
+#define GTEST_3_TYPENAMES_(T) typename T##0, typename T##1, typename T##2
+#define GTEST_4_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
+    typename T##3
+#define GTEST_5_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
+    typename T##3, typename T##4
+#define GTEST_6_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
+    typename T##3, typename T##4, typename T##5
+#define GTEST_7_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
+    typename T##3, typename T##4, typename T##5, typename T##6
+#define GTEST_8_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
+    typename T##3, typename T##4, typename T##5, typename T##6, typename T##7
+#define GTEST_9_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
+    typename T##3, typename T##4, typename T##5, typename T##6, \
+    typename T##7, typename T##8
+#define GTEST_10_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
+    typename T##3, typename T##4, typename T##5, typename T##6, \
+    typename T##7, typename T##8, typename T##9
+
+// In theory, defining stuff in the ::std namespace is undefined
+// behavior.  We can do this as we are playing the role of a standard
+// library vendor.
+namespace std {
+namespace tr1 {
+
+template <typename T0 = void, typename T1 = void, typename T2 = void,
+    typename T3 = void, typename T4 = void, typename T5 = void,
+    typename T6 = void, typename T7 = void, typename T8 = void,
+    typename T9 = void>
+class tuple;
+
+// Anything in namespace gtest_internal is Google Test's INTERNAL
+// IMPLEMENTATION DETAIL and MUST NOT BE USED DIRECTLY in user code.
+namespace gtest_internal {
+
+// ByRef<T>::type is T if T is a reference; otherwise it's const T&.
+template <typename T>
+struct ByRef { typedef const T& type; };  // NOLINT
+template <typename T>
+struct ByRef<T&> { typedef T& type; };  // NOLINT
+
+// A handy wrapper for ByRef.
+#define GTEST_BY_REF_(T) typename ::std::tr1::gtest_internal::ByRef<T>::type
+
+// AddRef<T>::type is T if T is a reference; otherwise it's T&.  This
+// is the same as tr1::add_reference<T>::type.
+template <typename T>
+struct AddRef { typedef T& type; };  // NOLINT
+template <typename T>
+struct AddRef<T&> { typedef T& type; };  // NOLINT
+
+// A handy wrapper for AddRef.
+#define GTEST_ADD_REF_(T) typename ::std::tr1::gtest_internal::AddRef<T>::type
+
+// A helper for implementing get<k>().
+template <int k> class Get;
+
+// A helper for implementing tuple_element<k, T>.  kIndexValid is true
+// iff k < the number of fields in tuple type T.
+template <bool kIndexValid, int kIndex, class Tuple>
+struct TupleElement;
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 0, GTEST_10_TUPLE_(T) > {
+  typedef T0 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 1, GTEST_10_TUPLE_(T) > {
+  typedef T1 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 2, GTEST_10_TUPLE_(T) > {
+  typedef T2 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 3, GTEST_10_TUPLE_(T) > {
+  typedef T3 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 4, GTEST_10_TUPLE_(T) > {
+  typedef T4 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 5, GTEST_10_TUPLE_(T) > {
+  typedef T5 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 6, GTEST_10_TUPLE_(T) > {
+  typedef T6 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 7, GTEST_10_TUPLE_(T) > {
+  typedef T7 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 8, GTEST_10_TUPLE_(T) > {
+  typedef T8 type;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct TupleElement<true, 9, GTEST_10_TUPLE_(T) > {
+  typedef T9 type;
+};
+
+}  // namespace gtest_internal
+
+template <>
+class tuple<> {
+ public:
+  tuple() {}
+  tuple(const tuple& /* t */)  {}
+  tuple& operator=(const tuple& /* t */) { return *this; }
+};
+
+template <GTEST_1_TYPENAMES_(T)>
+class GTEST_1_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0) : f0_(f0) {}
+
+  tuple(const tuple& t) : f0_(t.f0_) {}
+
+  template <GTEST_1_TYPENAMES_(U)>
+  tuple(const GTEST_1_TUPLE_(U)& t) : f0_(t.f0_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_1_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_1_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_1_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_1_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    return *this;
+  }
+
+  T0 f0_;
+};
+
+template <GTEST_2_TYPENAMES_(T)>
+class GTEST_2_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1) : f0_(f0),
+      f1_(f1) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_) {}
+
+  template <GTEST_2_TYPENAMES_(U)>
+  tuple(const GTEST_2_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_) {}
+  template <typename U0, typename U1>
+  tuple(const ::std::pair<U0, U1>& p) : f0_(p.first), f1_(p.second) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_2_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_2_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+  template <typename U0, typename U1>
+  tuple& operator=(const ::std::pair<U0, U1>& p) {
+    f0_ = p.first;
+    f1_ = p.second;
+    return *this;
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_2_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_2_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+};
+
+template <GTEST_3_TYPENAMES_(T)>
+class GTEST_3_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2) : f0_(f0), f1_(f1), f2_(f2) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_) {}
+
+  template <GTEST_3_TYPENAMES_(U)>
+  tuple(const GTEST_3_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_3_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_3_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_3_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_3_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+};
+
+template <GTEST_4_TYPENAMES_(T)>
+class GTEST_4_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_(), f3_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3) : f0_(f0), f1_(f1), f2_(f2),
+      f3_(f3) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_) {}
+
+  template <GTEST_4_TYPENAMES_(U)>
+  tuple(const GTEST_4_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
+      f3_(t.f3_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_4_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_4_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_4_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_4_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    f3_ = t.f3_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+  T3 f3_;
+};
+
+template <GTEST_5_TYPENAMES_(T)>
+class GTEST_5_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_(), f3_(), f4_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3,
+      GTEST_BY_REF_(T4) f4) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
+      f4_(t.f4_) {}
+
+  template <GTEST_5_TYPENAMES_(U)>
+  tuple(const GTEST_5_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
+      f3_(t.f3_), f4_(t.f4_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_5_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_5_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_5_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_5_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    f3_ = t.f3_;
+    f4_ = t.f4_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+  T3 f3_;
+  T4 f4_;
+};
+
+template <GTEST_6_TYPENAMES_(T)>
+class GTEST_6_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
+      GTEST_BY_REF_(T5) f5) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4),
+      f5_(f5) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
+      f4_(t.f4_), f5_(t.f5_) {}
+
+  template <GTEST_6_TYPENAMES_(U)>
+  tuple(const GTEST_6_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
+      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_6_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_6_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_6_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_6_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    f3_ = t.f3_;
+    f4_ = t.f4_;
+    f5_ = t.f5_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+  T3 f3_;
+  T4 f4_;
+  T5 f5_;
+};
+
+template <GTEST_7_TYPENAMES_(T)>
+class GTEST_7_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
+      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6) : f0_(f0), f1_(f1), f2_(f2),
+      f3_(f3), f4_(f4), f5_(f5), f6_(f6) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
+      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_) {}
+
+  template <GTEST_7_TYPENAMES_(U)>
+  tuple(const GTEST_7_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
+      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_7_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_7_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_7_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_7_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    f3_ = t.f3_;
+    f4_ = t.f4_;
+    f5_ = t.f5_;
+    f6_ = t.f6_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+  T3 f3_;
+  T4 f4_;
+  T5 f5_;
+  T6 f6_;
+};
+
+template <GTEST_8_TYPENAMES_(T)>
+class GTEST_8_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_(), f7_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
+      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6,
+      GTEST_BY_REF_(T7) f7) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4),
+      f5_(f5), f6_(f6), f7_(f7) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
+      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_) {}
+
+  template <GTEST_8_TYPENAMES_(U)>
+  tuple(const GTEST_8_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
+      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_8_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_8_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_8_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_8_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    f3_ = t.f3_;
+    f4_ = t.f4_;
+    f5_ = t.f5_;
+    f6_ = t.f6_;
+    f7_ = t.f7_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+  T3 f3_;
+  T4 f4_;
+  T5 f5_;
+  T6 f6_;
+  T7 f7_;
+};
+
+template <GTEST_9_TYPENAMES_(T)>
+class GTEST_9_TUPLE_(T) {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_(), f7_(), f8_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
+      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6, GTEST_BY_REF_(T7) f7,
+      GTEST_BY_REF_(T8) f8) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4),
+      f5_(f5), f6_(f6), f7_(f7), f8_(f8) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
+      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_) {}
+
+  template <GTEST_9_TYPENAMES_(U)>
+  tuple(const GTEST_9_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
+      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_9_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_9_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_9_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_9_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    f3_ = t.f3_;
+    f4_ = t.f4_;
+    f5_ = t.f5_;
+    f6_ = t.f6_;
+    f7_ = t.f7_;
+    f8_ = t.f8_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+  T3 f3_;
+  T4 f4_;
+  T5 f5_;
+  T6 f6_;
+  T7 f7_;
+  T8 f8_;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+class tuple {
+ public:
+  template <int k> friend class gtest_internal::Get;
+
+  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_(), f7_(), f8_(),
+      f9_() {}
+
+  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
+      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
+      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6, GTEST_BY_REF_(T7) f7,
+      GTEST_BY_REF_(T8) f8, GTEST_BY_REF_(T9) f9) : f0_(f0), f1_(f1), f2_(f2),
+      f3_(f3), f4_(f4), f5_(f5), f6_(f6), f7_(f7), f8_(f8), f9_(f9) {}
+
+  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
+      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_), f9_(t.f9_) {}
+
+  template <GTEST_10_TYPENAMES_(U)>
+  tuple(const GTEST_10_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
+      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_),
+      f9_(t.f9_) {}
+
+  tuple& operator=(const tuple& t) { return CopyFrom(t); }
+
+  template <GTEST_10_TYPENAMES_(U)>
+  tuple& operator=(const GTEST_10_TUPLE_(U)& t) {
+    return CopyFrom(t);
+  }
+
+  GTEST_DECLARE_TUPLE_AS_FRIEND_
+
+  template <GTEST_10_TYPENAMES_(U)>
+  tuple& CopyFrom(const GTEST_10_TUPLE_(U)& t) {
+    f0_ = t.f0_;
+    f1_ = t.f1_;
+    f2_ = t.f2_;
+    f3_ = t.f3_;
+    f4_ = t.f4_;
+    f5_ = t.f5_;
+    f6_ = t.f6_;
+    f7_ = t.f7_;
+    f8_ = t.f8_;
+    f9_ = t.f9_;
+    return *this;
+  }
+
+  T0 f0_;
+  T1 f1_;
+  T2 f2_;
+  T3 f3_;
+  T4 f4_;
+  T5 f5_;
+  T6 f6_;
+  T7 f7_;
+  T8 f8_;
+  T9 f9_;
+};
+
+// 6.1.3.2 Tuple creation functions.
+
+// Known limitations: we don't support passing an
+// std::tr1::reference_wrapper<T> to make_tuple().  And we don't
+// implement tie().
+
+inline tuple<> make_tuple() { return tuple<>(); }
+
+template <GTEST_1_TYPENAMES_(T)>
+inline GTEST_1_TUPLE_(T) make_tuple(const T0& f0) {
+  return GTEST_1_TUPLE_(T)(f0);
+}
+
+template <GTEST_2_TYPENAMES_(T)>
+inline GTEST_2_TUPLE_(T) make_tuple(const T0& f0, const T1& f1) {
+  return GTEST_2_TUPLE_(T)(f0, f1);
+}
+
+template <GTEST_3_TYPENAMES_(T)>
+inline GTEST_3_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2) {
+  return GTEST_3_TUPLE_(T)(f0, f1, f2);
+}
+
+template <GTEST_4_TYPENAMES_(T)>
+inline GTEST_4_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
+    const T3& f3) {
+  return GTEST_4_TUPLE_(T)(f0, f1, f2, f3);
+}
+
+template <GTEST_5_TYPENAMES_(T)>
+inline GTEST_5_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
+    const T3& f3, const T4& f4) {
+  return GTEST_5_TUPLE_(T)(f0, f1, f2, f3, f4);
+}
+
+template <GTEST_6_TYPENAMES_(T)>
+inline GTEST_6_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
+    const T3& f3, const T4& f4, const T5& f5) {
+  return GTEST_6_TUPLE_(T)(f0, f1, f2, f3, f4, f5);
+}
+
+template <GTEST_7_TYPENAMES_(T)>
+inline GTEST_7_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
+    const T3& f3, const T4& f4, const T5& f5, const T6& f6) {
+  return GTEST_7_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6);
+}
+
+template <GTEST_8_TYPENAMES_(T)>
+inline GTEST_8_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
+    const T3& f3, const T4& f4, const T5& f5, const T6& f6, const T7& f7) {
+  return GTEST_8_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6, f7);
+}
+
+template <GTEST_9_TYPENAMES_(T)>
+inline GTEST_9_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
+    const T3& f3, const T4& f4, const T5& f5, const T6& f6, const T7& f7,
+    const T8& f8) {
+  return GTEST_9_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6, f7, f8);
+}
+
+template <GTEST_10_TYPENAMES_(T)>
+inline GTEST_10_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
+    const T3& f3, const T4& f4, const T5& f5, const T6& f6, const T7& f7,
+    const T8& f8, const T9& f9) {
+  return GTEST_10_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6, f7, f8, f9);
+}
+
+// 6.1.3.3 Tuple helper classes.
+
+template <typename Tuple> struct tuple_size;
+
+template <GTEST_0_TYPENAMES_(T)>
+struct tuple_size<GTEST_0_TUPLE_(T) > {
+  static const int value = 0;
+};
+
+template <GTEST_1_TYPENAMES_(T)>
+struct tuple_size<GTEST_1_TUPLE_(T) > {
+  static const int value = 1;
+};
+
+template <GTEST_2_TYPENAMES_(T)>
+struct tuple_size<GTEST_2_TUPLE_(T) > {
+  static const int value = 2;
+};
+
+template <GTEST_3_TYPENAMES_(T)>
+struct tuple_size<GTEST_3_TUPLE_(T) > {
+  static const int value = 3;
+};
+
+template <GTEST_4_TYPENAMES_(T)>
+struct tuple_size<GTEST_4_TUPLE_(T) > {
+  static const int value = 4;
+};
+
+template <GTEST_5_TYPENAMES_(T)>
+struct tuple_size<GTEST_5_TUPLE_(T) > {
+  static const int value = 5;
+};
+
+template <GTEST_6_TYPENAMES_(T)>
+struct tuple_size<GTEST_6_TUPLE_(T) > {
+  static const int value = 6;
+};
+
+template <GTEST_7_TYPENAMES_(T)>
+struct tuple_size<GTEST_7_TUPLE_(T) > {
+  static const int value = 7;
+};
+
+template <GTEST_8_TYPENAMES_(T)>
+struct tuple_size<GTEST_8_TUPLE_(T) > {
+  static const int value = 8;
+};
+
+template <GTEST_9_TYPENAMES_(T)>
+struct tuple_size<GTEST_9_TUPLE_(T) > {
+  static const int value = 9;
+};
+
+template <GTEST_10_TYPENAMES_(T)>
+struct tuple_size<GTEST_10_TUPLE_(T) > {
+  static const int value = 10;
+};
+
+template <int k, class Tuple>
+struct tuple_element {
+  typedef typename gtest_internal::TupleElement<
+      k < (tuple_size<Tuple>::value), k, Tuple>::type type;
+};
+
+#define GTEST_TUPLE_ELEMENT_(k, Tuple) typename tuple_element<k, Tuple >::type
+
+// 6.1.3.4 Element access.
+
+namespace gtest_internal {
+
+template <>
+class Get<0> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(0, Tuple))
+  Field(Tuple& t) { return t.f0_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(0, Tuple))
+  ConstField(const Tuple& t) { return t.f0_; }
+};
+
+template <>
+class Get<1> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(1, Tuple))
+  Field(Tuple& t) { return t.f1_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(1, Tuple))
+  ConstField(const Tuple& t) { return t.f1_; }
+};
+
+template <>
+class Get<2> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(2, Tuple))
+  Field(Tuple& t) { return t.f2_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(2, Tuple))
+  ConstField(const Tuple& t) { return t.f2_; }
+};
+
+template <>
+class Get<3> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(3, Tuple))
+  Field(Tuple& t) { return t.f3_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(3, Tuple))
+  ConstField(const Tuple& t) { return t.f3_; }
+};
+
+template <>
+class Get<4> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(4, Tuple))
+  Field(Tuple& t) { return t.f4_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(4, Tuple))
+  ConstField(const Tuple& t) { return t.f4_; }
+};
+
+template <>
+class Get<5> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(5, Tuple))
+  Field(Tuple& t) { return t.f5_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(5, Tuple))
+  ConstField(const Tuple& t) { return t.f5_; }
+};
+
+template <>
+class Get<6> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(6, Tuple))
+  Field(Tuple& t) { return t.f6_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(6, Tuple))
+  ConstField(const Tuple& t) { return t.f6_; }
+};
+
+template <>
+class Get<7> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(7, Tuple))
+  Field(Tuple& t) { return t.f7_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(7, Tuple))
+  ConstField(const Tuple& t) { return t.f7_; }
+};
+
+template <>
+class Get<8> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(8, Tuple))
+  Field(Tuple& t) { return t.f8_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(8, Tuple))
+  ConstField(const Tuple& t) { return t.f8_; }
+};
+
+template <>
+class Get<9> {
+ public:
+  template <class Tuple>
+  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(9, Tuple))
+  Field(Tuple& t) { return t.f9_; }  // NOLINT
+
+  template <class Tuple>
+  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(9, Tuple))
+  ConstField(const Tuple& t) { return t.f9_; }
+};
+
+}  // namespace gtest_internal
+
+template <int k, GTEST_10_TYPENAMES_(T)>
+GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(k, GTEST_10_TUPLE_(T)))
+get(GTEST_10_TUPLE_(T)& t) {
+  return gtest_internal::Get<k>::Field(t);
+}
+
+template <int k, GTEST_10_TYPENAMES_(T)>
+GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(k,  GTEST_10_TUPLE_(T)))
+get(const GTEST_10_TUPLE_(T)& t) {
+  return gtest_internal::Get<k>::ConstField(t);
+}
+
+// 6.1.3.5 Relational operators
+
+// We only implement == and !=, as we don't have a need for the rest yet.
+
+namespace gtest_internal {
+
+// SameSizeTuplePrefixComparator<k, k>::Eq(t1, t2) returns true if the
+// first k fields of t1 equals the first k fields of t2.
+// SameSizeTuplePrefixComparator(k1, k2) would be a compiler error if
+// k1 != k2.
+template <int kSize1, int kSize2>
+struct SameSizeTuplePrefixComparator;
+
+template <>
+struct SameSizeTuplePrefixComparator<0, 0> {
+  template <class Tuple1, class Tuple2>
+  static bool Eq(const Tuple1& /* t1 */, const Tuple2& /* t2 */) {
+    return true;
+  }
+};
+
+template <int k>
+struct SameSizeTuplePrefixComparator<k, k> {
+  template <class Tuple1, class Tuple2>
+  static bool Eq(const Tuple1& t1, const Tuple2& t2) {
+    return SameSizeTuplePrefixComparator<k - 1, k - 1>::Eq(t1, t2) &&
+        ::std::tr1::get<k - 1>(t1) == ::std::tr1::get<k - 1>(t2);
+  }
+};
+
+}  // namespace gtest_internal
+
+template <GTEST_10_TYPENAMES_(T), GTEST_10_TYPENAMES_(U)>
+inline bool operator==(const GTEST_10_TUPLE_(T)& t,
+                       const GTEST_10_TUPLE_(U)& u) {
+  return gtest_internal::SameSizeTuplePrefixComparator<
+      tuple_size<GTEST_10_TUPLE_(T) >::value,
+      tuple_size<GTEST_10_TUPLE_(U) >::value>::Eq(t, u);
+}
+
+template <GTEST_10_TYPENAMES_(T), GTEST_10_TYPENAMES_(U)>
+inline bool operator!=(const GTEST_10_TUPLE_(T)& t,
+                       const GTEST_10_TUPLE_(U)& u) { return !(t == u); }
+
+// 6.1.4 Pairs.
+// Unimplemented.
+
+}  // namespace tr1
+}  // namespace std
+
+#undef GTEST_0_TUPLE_
+#undef GTEST_1_TUPLE_
+#undef GTEST_2_TUPLE_
+#undef GTEST_3_TUPLE_
+#undef GTEST_4_TUPLE_
+#undef GTEST_5_TUPLE_
+#undef GTEST_6_TUPLE_
+#undef GTEST_7_TUPLE_
+#undef GTEST_8_TUPLE_
+#undef GTEST_9_TUPLE_
+#undef GTEST_10_TUPLE_
+
+#undef GTEST_0_TYPENAMES_
+#undef GTEST_1_TYPENAMES_
+#undef GTEST_2_TYPENAMES_
+#undef GTEST_3_TYPENAMES_
+#undef GTEST_4_TYPENAMES_
+#undef GTEST_5_TYPENAMES_
+#undef GTEST_6_TYPENAMES_
+#undef GTEST_7_TYPENAMES_
+#undef GTEST_8_TYPENAMES_
+#undef GTEST_9_TYPENAMES_
+#undef GTEST_10_TYPENAMES_
+
+#undef GTEST_DECLARE_TUPLE_AS_FRIEND_
+#undef GTEST_BY_REF_
+#undef GTEST_ADD_REF_
+#undef GTEST_TUPLE_ELEMENT_
+
+#endif  // GTEST_INCLUDE_GTEST_INTERNAL_GTEST_TUPLE_H_
+# elif GTEST_ENV_HAS_STD_TUPLE_
+#  include <tuple>
+// C++11 puts its tuple into the ::std namespace rather than
+// ::std::tr1.  gtest expects tuple to live in ::std::tr1, so put it there.
+// This causes undefined behavior, but supported compilers react in
+// the way we intend.
+namespace std {
+namespace tr1 {
+using ::std::get;
+using ::std::make_tuple;
+using ::std::tuple;
+using ::std::tuple_element;
+using ::std::tuple_size;
+}
+}
+
+# elif GTEST_OS_SYMBIAN
+
+// On Symbian, BOOST_HAS_TR1_TUPLE causes Boost's TR1 tuple library to
+// use STLport's tuple implementation, which unfortunately doesn't
+// work as the copy of STLport distributed with Symbian is incomplete.
+// By making sure BOOST_HAS_TR1_TUPLE is undefined, we force Boost to
+// use its own tuple implementation.
+#  ifdef BOOST_HAS_TR1_TUPLE
+#   undef BOOST_HAS_TR1_TUPLE
+#  endif  // BOOST_HAS_TR1_TUPLE
+
+// This prevents <boost/tr1/detail/config.hpp>, which defines
+// BOOST_HAS_TR1_TUPLE, from being #included by Boost's <tuple>.
+#  define BOOST_TR1_DETAIL_CONFIG_HPP_INCLUDED
+#  include <tuple>  // IWYU pragma: export  // NOLINT
+
+# elif defined(__GNUC__) && (GTEST_GCC_VER_ >= 40000)
+// GCC 4.0+ implements tr1/tuple in the <tr1/tuple> header.  This does
+// not conform to the TR1 spec, which requires the header to be <tuple>.
+
+#  if !GTEST_HAS_RTTI && GTEST_GCC_VER_ < 40302
+// Until version 4.3.2, gcc has a bug that causes <tr1/functional>,
+// which is #included by <tr1/tuple>, to not compile when RTTI is
+// disabled.  _TR1_FUNCTIONAL is the header guard for
+// <tr1/functional>.  Hence the following #define is a hack to prevent
+// <tr1/functional> from being included.
+#   define _TR1_FUNCTIONAL 1
+#   include <tr1/tuple>
+#   undef _TR1_FUNCTIONAL  // Allows the user to #include
+                        // <tr1/functional> if he chooses to.
+#  else
+#   include <tr1/tuple>  // NOLINT
+#  endif  // !GTEST_HAS_RTTI && GTEST_GCC_VER_ < 40302
+
+# else
+// If the compiler is not GCC 4.0+, we assume the user is using a
+// spec-conforming TR1 implementation.
+#  include <tuple>  // IWYU pragma: export  // NOLINT
+# endif  // GTEST_USE_OWN_TR1_TUPLE
+
+#endif  // GTEST_HAS_TR1_TUPLE
+
+// Determines whether clone(2) is supported.
+// Usually it will only be available on Linux, excluding
+// Linux on the Itanium architecture.
+// Also see http://linux.die.net/man/2/clone.
+#ifndef GTEST_HAS_CLONE
+// The user didn't tell us, so we need to figure it out.
+
+# if GTEST_OS_LINUX && !defined(__ia64__)
+#  if GTEST_OS_LINUX_ANDROID
+// On Android, clone() is only available on ARM starting with Gingerbread.
+#    if defined(__arm__) && __ANDROID_API__ >= 9
+#     define GTEST_HAS_CLONE 1
+#    else
+#     define GTEST_HAS_CLONE 0
+#    endif
+#  else
+#   define GTEST_HAS_CLONE 1
+#  endif
+# else
+#  define GTEST_HAS_CLONE 0
+# endif  // GTEST_OS_LINUX && !defined(__ia64__)
+
+#endif  // GTEST_HAS_CLONE
+
+// Determines whether to support stream redirection. This is used to test
+// output correctness and to implement death tests.
+#ifndef GTEST_HAS_STREAM_REDIRECTION
+// By default, we assume that stream redirection is supported on all
+// platforms except known mobile ones.
+# if GTEST_OS_WINDOWS_MOBILE || GTEST_OS_SYMBIAN || \
+    GTEST_OS_WINDOWS_PHONE || GTEST_OS_WINDOWS_RT
+#  define GTEST_HAS_STREAM_REDIRECTION 0
+# else
+#  define GTEST_HAS_STREAM_REDIRECTION 1
+# endif  // !GTEST_OS_WINDOWS_MOBILE && !GTEST_OS_SYMBIAN
+#endif  // GTEST_HAS_STREAM_REDIRECTION
+
+// Determines whether to support death tests.
+// Google Test does not support death tests for VC 7.1 and earlier as
+// abort() in a VC 7.1 application compiled as GUI in debug config
+// pops up a dialog window that cannot be suppressed programmatically.
+#if (GTEST_OS_LINUX || GTEST_OS_CYGWIN || GTEST_OS_SOLARIS || \
+     (GTEST_OS_MAC && !GTEST_OS_IOS) || \
+     (GTEST_OS_WINDOWS_DESKTOP && _MSC_VER >= 1400) || \
+     GTEST_OS_WINDOWS_MINGW || GTEST_OS_AIX || GTEST_OS_HPUX || \
+     GTEST_OS_OPENBSD || GTEST_OS_QNX || GTEST_OS_FREEBSD)
+# define GTEST_HAS_DEATH_TEST 1
+#endif
+
+// We don't support MSVC 7.1 with exceptions disabled now.  Therefore
+// all the compilers we care about are adequate for supporting
+// value-parameterized tests.
+#define GTEST_HAS_PARAM_TEST 1
+
+// Determines whether to support type-driven tests.
+
+// Typed tests need <typeinfo> and variadic macros, which GCC, VC++ 8.0,
+// Sun Pro CC, IBM Visual Age, and HP aCC support.
+#if defined(__GNUC__) || (_MSC_VER >= 1400) || defined(__SUNPRO_CC) || \
+    defined(__IBMCPP__) || defined(__HP_aCC)
+# define GTEST_HAS_TYPED_TEST 1
+# define GTEST_HAS_TYPED_TEST_P 1
+#endif
+
+// Determines whether to support Combine(). This only makes sense when
+// value-parameterized tests are enabled.  The implementation doesn't
+// work on Sun Studio since it doesn't understand templated conversion
+// operators.
+#if GTEST_HAS_PARAM_TEST && GTEST_HAS_TR1_TUPLE && !defined(__SUNPRO_CC)
+# define GTEST_HAS_COMBINE 1
+#endif
+
+// Determines whether the system compiler uses UTF-16 for encoding wide strings.
+#define GTEST_WIDE_STRING_USES_UTF16_ \
+    (GTEST_OS_WINDOWS || GTEST_OS_CYGWIN || GTEST_OS_SYMBIAN || GTEST_OS_AIX)
+
+// Determines whether test results can be streamed to a socket.
+#if GTEST_OS_LINUX
+# define GTEST_CAN_STREAM_RESULTS_ 1
+#endif
+
+// Defines some utility macros.
+
+// The GNU compiler emits a warning if nested "if" statements are followed by
+// an "else" statement and braces are not used to explicitly disambiguate the
+// "else" binding.  This leads to problems with code like:
+//
+//   if (gate)
+//     ASSERT_*(condition) << "Some message";
+//
+// The "switch (0) case 0:" idiom is used to suppress this.
+#ifdef __INTEL_COMPILER
+# define GTEST_AMBIGUOUS_ELSE_BLOCKER_
+#else
+# define GTEST_AMBIGUOUS_ELSE_BLOCKER_ switch (0) case 0: default:  // NOLINT
+#endif
+
+// Use this annotation at the end of a struct/class definition to
+// prevent the compiler from optimizing away instances that are never
+// used.  This is useful when all interesting logic happens inside the
+// c'tor and / or d'tor.  Example:
+//
+//   struct Foo {
+//     Foo() { ... }
+//   } GTEST_ATTRIBUTE_UNUSED_;
+//
+// Also use it after a variable or parameter declaration to tell the
+// compiler the variable/parameter does not have to be used.
+#if defined(__GNUC__) && !defined(COMPILER_ICC)
+# define GTEST_ATTRIBUTE_UNUSED_ __attribute__ ((unused))
+#elif defined(__clang__)
+# if __has_attribute(unused)
+#  define GTEST_ATTRIBUTE_UNUSED_ __attribute__ ((unused))
+# endif
+#endif
+#ifndef GTEST_ATTRIBUTE_UNUSED_
+# define GTEST_ATTRIBUTE_UNUSED_
+#endif
+
+// A macro to disallow operator=
+// This should be used in the private: declarations for a class.
+#define GTEST_DISALLOW_ASSIGN_(type)\
+  void operator=(type const &)
+
+// A macro to disallow copy constructor and operator=
+// This should be used in the private: declarations for a class.
+#define GTEST_DISALLOW_COPY_AND_ASSIGN_(type)\
+  type(type const &);\
+  GTEST_DISALLOW_ASSIGN_(type)
+
+// Tell the compiler to warn about unused return values for functions declared
+// with this macro.  The macro should be used on function declarations
+// following the argument list:
+//
+//   Sprocket* AllocateSprocket() GTEST_MUST_USE_RESULT_;
+#if defined(__GNUC__) && (GTEST_GCC_VER_ >= 30400) && !defined(COMPILER_ICC)
+# define GTEST_MUST_USE_RESULT_ __attribute__ ((warn_unused_result))
+#else
+# define GTEST_MUST_USE_RESULT_
+#endif  // __GNUC__ && (GTEST_GCC_VER_ >= 30400) && !COMPILER_ICC
+
+// MS C++ compiler emits warning when a conditional expression is compile time
+// constant. In some contexts this warning is false positive and needs to be
+// suppressed. Use the following two macros in such cases:
+//
+// GTEST_INTENTIONAL_CONST_COND_PUSH_()
+// while (true) {
+// GTEST_INTENTIONAL_CONST_COND_POP_()
+// }
+# define GTEST_INTENTIONAL_CONST_COND_PUSH_() \
+    GTEST_DISABLE_MSC_WARNINGS_PUSH_(4127)
+# define GTEST_INTENTIONAL_CONST_COND_POP_() \
+    GTEST_DISABLE_MSC_WARNINGS_POP_()
+
+// Determine whether the compiler supports Microsoft's Structured Exception
+// Handling.  This is supported by several Windows compilers but generally
+// does not exist on any other system.
+#ifndef GTEST_HAS_SEH
+// The user didn't tell us, so we need to figure it out.
+
+# if defined(_MSC_VER) || defined(__BORLANDC__)
+// These two compilers are known to support SEH.
+#  define GTEST_HAS_SEH 1
+# else
+// Assume no SEH.
+#  define GTEST_HAS_SEH 0
+# endif
+
+#define GTEST_IS_THREADSAFE \
+    (GTEST_HAS_MUTEX_AND_THREAD_LOCAL_ \
+     || (GTEST_OS_WINDOWS && !GTEST_OS_WINDOWS_PHONE && !GTEST_OS_WINDOWS_RT) \
+     || GTEST_HAS_PTHREAD)
+
+#endif  // GTEST_HAS_SEH
+
+#ifdef _MSC_VER
+# if GTEST_LINKED_AS_SHARED_LIBRARY
+#  define GTEST_API_ __declspec(dllimport)
+# elif GTEST_CREATE_SHARED_LIBRARY
+#  define GTEST_API_ __declspec(dllexport)
+# endif
+#elif __GNUC__ >= 4 || defined(__clang__)
+# define GTEST_API_ __attribute__((visibility ("default")))
+#endif // _MSC_VER
+
+#ifndef GTEST_API_
+# define GTEST_API_
+#endif
+
+#ifdef __GNUC__
+// Ask the compiler to never inline a given function.
+# define GTEST_NO_INLINE_ __attribute__((noinline))
+#else
+# define GTEST_NO_INLINE_
+#endif
+
+// _LIBCPP_VERSION is defined by the libc++ library from the LLVM project.
+#if defined(__GLIBCXX__) || defined(_LIBCPP_VERSION)
+# define GTEST_HAS_CXXABI_H_ 1
+#else
+# define GTEST_HAS_CXXABI_H_ 0
+#endif
+
+// A function level attribute to disable checking for use of uninitialized
+// memory when built with MemorySanitizer.
+#if defined(__clang__)
+# if __has_feature(memory_sanitizer)
+#  define GTEST_ATTRIBUTE_NO_SANITIZE_MEMORY_ \
+       __attribute__((no_sanitize_memory))
+# else
+#  define GTEST_ATTRIBUTE_NO_SANITIZE_MEMORY_
+# endif  // __has_feature(memory_sanitizer)
+#else
+# define GTEST_ATTRIBUTE_NO_SANITIZE_MEMORY_
+#endif  // __clang__
+
+// A function level attribute to disable AddressSanitizer instrumentation.
+#if defined(__clang__)
+# if __has_feature(address_sanitizer)
+#  define GTEST_ATTRIBUTE_NO_SANITIZE_ADDRESS_ \
+       __attribute__((no_sanitize_address))
+# else
+#  define GTEST_ATTRIBUTE_NO_SANITIZE_ADDRESS_
+# endif  // __has_feature(address_sanitizer)
+#else
+# define GTEST_ATTRIBUTE_NO_SANITIZE_ADDRESS_
+#endif  // __clang__
+
+// A function level attribute to disable ThreadSanitizer instrumentation.
+#if defined(__clang__)
+# if __has_feature(thread_sanitizer)
+#  define GTEST_ATTRIBUTE_NO_SANITIZE_THREAD_ \
+       __attribute__((no_sanitize_thread))
+# else
+#  define GTEST_ATTRIBUTE_NO_SANITIZE_THREAD_
+# endif  // __has_feature(thread_sanitizer)
+#else
+# define GTEST_ATTRIBUTE_NO_SANITIZE_THREAD_
+#endif  // __clang__
+
+namespace testing {
+
+class Message;
+
+#if defined(GTEST_TUPLE_NAMESPACE_)
+// Import tuple and friends into the ::testing namespace.
+// It is part of our interface, having them in ::testing allows us to change
+// their types as needed.
+using GTEST_TUPLE_NAMESPACE_::get;
+using GTEST_TUPLE_NAMESPACE_::make_tuple;
+using GTEST_TUPLE_NAMESPACE_::tuple;
+using GTEST_TUPLE_NAMESPACE_::tuple_size;
+using GTEST_TUPLE_NAMESPACE_::tuple_element;
+#endif  // defined(GTEST_TUPLE_NAMESPACE_)
+
+namespace internal {
+
+// A secret type that Google Test users don't know about.  It has no
+// definition on purpose.  Therefore it's impossible to create a
+// Secret object, which is what we want.
+class Secret;
+
+// The GTEST_COMPILE_ASSERT_ macro can be used to verify that a compile time
+// expression is true. For example, you could use it to verify the
+// size of a static array:
+//
+//   GTEST_COMPILE_ASSERT_(GTEST_ARRAY_SIZE_(names) == NUM_NAMES,
+//                         names_incorrect_size);
+//
+// or to make sure a struct is smaller than a certain size:
+//
+//   GTEST_COMPILE_ASSERT_(sizeof(foo) < 128, foo_too_large);
+//
+// The second argument to the macro is the name of the variable. If
+// the expression is false, most compilers will issue a warning/error
+// containing the name of the variable.
+
+#if GTEST_LANG_CXX11
+# define GTEST_COMPILE_ASSERT_(expr, msg) static_assert(expr, #msg)
+#else  // !GTEST_LANG_CXX11
+template <bool>
+  struct CompileAssert {
+};
+
+# define GTEST_COMPILE_ASSERT_(expr, msg) \
+  typedef ::testing::internal::CompileAssert<(static_cast<bool>(expr))> \
+      msg[static_cast<bool>(expr) ? 1 : -1] GTEST_ATTRIBUTE_UNUSED_
+#endif  // !GTEST_LANG_CXX11
+
+// Implementation details of GTEST_COMPILE_ASSERT_:
+//
+// (In C++11, we simply use static_assert instead of the following)
+//
+// - GTEST_COMPILE_ASSERT_ works by defining an array type that has -1
+//   elements (and thus is invalid) when the expression is false.
+//
+// - The simpler definition
+//
+//    #define GTEST_COMPILE_ASSERT_(expr, msg) typedef char msg[(expr) ? 1 : -1]
+//
+//   does not work, as gcc supports variable-length arrays whose sizes
+//   are determined at run-time (this is gcc's extension and not part
+//   of the C++ standard).  As a result, gcc fails to reject the
+//   following code with the simple definition:
+//
+//     int foo;
+//     GTEST_COMPILE_ASSERT_(foo, msg); // not supposed to compile as foo is
+//                                      // not a compile-time constant.
+//
+// - By using the type CompileAssert<(bool(expr))>, we ensures that
+//   expr is a compile-time constant.  (Template arguments must be
+//   determined at compile-time.)
+//
+// - The outter parentheses in CompileAssert<(bool(expr))> are necessary
+//   to work around a bug in gcc 3.4.4 and 4.0.1.  If we had written
+//
+//     CompileAssert<bool(expr)>
+//
+//   instead, these compilers will refuse to compile
+//
+//     GTEST_COMPILE_ASSERT_(5 > 0, some_message);
+//
+//   (They seem to think the ">" in "5 > 0" marks the end of the
+//   template argument list.)
+//
+// - The array size is (bool(expr) ? 1 : -1), instead of simply
+//
+//     ((expr) ? 1 : -1).
+//
+//   This is to avoid running into a bug in MS VC 7.1, which
+//   causes ((0.0) ? 1 : -1) to incorrectly evaluate to 1.
+
+// StaticAssertTypeEqHelper is used by StaticAssertTypeEq defined in gtest.h.
+//
+// This template is declared, but intentionally undefined.
+template <typename T1, typename T2>
+struct StaticAssertTypeEqHelper;
+
+template <typename T>
+struct StaticAssertTypeEqHelper<T, T> {
+  enum { value = true };
+};
+
+// Evaluates to the number of elements in 'array'.
+#define GTEST_ARRAY_SIZE_(array) (sizeof(array) / sizeof(array[0]))
+
+#if GTEST_HAS_GLOBAL_STRING
+typedef ::string string;
+#else
+typedef ::std::string string;
+#endif  // GTEST_HAS_GLOBAL_STRING
+
+#if GTEST_HAS_GLOBAL_WSTRING
+typedef ::wstring wstring;
+#elif GTEST_HAS_STD_WSTRING
+typedef ::std::wstring wstring;
+#endif  // GTEST_HAS_GLOBAL_WSTRING
+
+// A helper for suppressing warnings on constant condition.  It just
+// returns 'condition'.
+GTEST_API_ bool IsTrue(bool condition);
+
+// Defines scoped_ptr.
+
+// This implementation of scoped_ptr is PARTIAL - it only contains
+// enough stuff to satisfy Google Test's need.
+template <typename T>
+class scoped_ptr {
+ public:
+  typedef T element_type;
+
+  explicit scoped_ptr(T* p = NULL) : ptr_(p) {}
+  ~scoped_ptr() { reset(); }
+
+  T& operator*() const { return *ptr_; }
+  T* operator->() const { return ptr_; }
+  T* get() const { return ptr_; }
+
+  T* release() {
+    T* const ptr = ptr_;
+    ptr_ = NULL;
+    return ptr;
+  }
+
+  void reset(T* p = NULL) {
+    if (p != ptr_) {
+      if (IsTrue(sizeof(T) > 0)) {  // Makes sure T is a complete type.
+        delete ptr_;
+      }
+      ptr_ = p;
+    }
+  }
+
+  friend void swap(scoped_ptr& a, scoped_ptr& b) {
+    using std::swap;
+    swap(a.ptr_, b.ptr_);
+  }
+
+ private:
+  T* ptr_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(scoped_ptr);
+};
+
+// Defines RE.
+
+// A simple C++ wrapper for <regex.h>.  It uses the POSIX Extended
+// Regular Expression syntax.
+class GTEST_API_ RE {
+ public:
+  // A copy constructor is required by the Standard to initialize object
+  // references from r-values.
+  RE(const RE& other) { Init(other.pattern()); }
+
+  // Constructs an RE from a string.
+  RE(const ::std::string& regex) { Init(regex.c_str()); }  // NOLINT
+
+#if GTEST_HAS_GLOBAL_STRING
+
+  RE(const ::string& regex) { Init(regex.c_str()); }  // NOLINT
+
+#endif  // GTEST_HAS_GLOBAL_STRING
+
+  RE(const char* regex) { Init(regex); }  // NOLINT
+  ~RE();
+
+  // Returns the string representation of the regex.
+  const char* pattern() const { return pattern_; }
+
+  // FullMatch(str, re) returns true iff regular expression re matches
+  // the entire str.
+  // PartialMatch(str, re) returns true iff regular expression re
+  // matches a substring of str (including str itself).
+  //
+  // TODO(wan@google.com): make FullMatch() and PartialMatch() work
+  // when str contains NUL characters.
+  static bool FullMatch(const ::std::string& str, const RE& re) {
+    return FullMatch(str.c_str(), re);
+  }
+  static bool PartialMatch(const ::std::string& str, const RE& re) {
+    return PartialMatch(str.c_str(), re);
+  }
+
+#if GTEST_HAS_GLOBAL_STRING
+
+  static bool FullMatch(const ::string& str, const RE& re) {
+    return FullMatch(str.c_str(), re);
+  }
+  static bool PartialMatch(const ::string& str, const RE& re) {
+    return PartialMatch(str.c_str(), re);
+  }
+
+#endif  // GTEST_HAS_GLOBAL_STRING
+
+  static bool FullMatch(const char* str, const RE& re);
+  static bool PartialMatch(const char* str, const RE& re);
+
+ private:
+  void Init(const char* regex);
+
+  // We use a const char* instead of an std::string, as Google Test used to be
+  // used where std::string is not available.  TODO(wan@google.com): change to
+  // std::string.
+  const char* pattern_;
+  bool is_valid_;
+
+#if GTEST_USES_POSIX_RE
+
+  regex_t full_regex_;     // For FullMatch().
+  regex_t partial_regex_;  // For PartialMatch().
+
+#else  // GTEST_USES_SIMPLE_RE
+
+  const char* full_pattern_;  // For FullMatch();
+
+#endif
+
+  GTEST_DISALLOW_ASSIGN_(RE);
+};
+
+// Formats a source file path and a line number as they would appear
+// in an error message from the compiler used to compile this code.
+GTEST_API_ ::std::string FormatFileLocation(const char* file, int line);
+
+// Formats a file location for compiler-independent XML output.
+// Although this function is not platform dependent, we put it next to
+// FormatFileLocation in order to contrast the two functions.
+GTEST_API_ ::std::string FormatCompilerIndependentFileLocation(const char* file,
+                                                               int line);
+
+// Defines logging utilities:
+//   GTEST_LOG_(severity) - logs messages at the specified severity level. The
+//                          message itself is streamed into the macro.
+//   LogToStderr()  - directs all log messages to stderr.
+//   FlushInfoLog() - flushes informational log messages.
+
+enum GTestLogSeverity {
+  GTEST_INFO,
+  GTEST_WARNING,
+  GTEST_ERROR,
+  GTEST_FATAL
+};
+
+// Formats log entry severity, provides a stream object for streaming the
+// log message, and terminates the message with a newline when going out of
+// scope.
+class GTEST_API_ GTestLog {
+ public:
+  GTestLog(GTestLogSeverity severity, const char* file, int line);
+
+  // Flushes the buffers and, if severity is GTEST_FATAL, aborts the program.
+  ~GTestLog();
+
+  ::std::ostream& GetStream() { return ::std::cerr; }
+
+ private:
+  const GTestLogSeverity severity_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(GTestLog);
+};
+
+#if !defined(GTEST_LOG_)
+
+#

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/50] [abbrv] hadoop git commit: YARN-5559. Analyse 2.8.0/3.0.0 jdiff reports and fix any issues. Contributed by Akira Ajisaka & Wangda Tan

Posted by st...@apache.org.
YARN-5559. Analyse 2.8.0/3.0.0 jdiff reports and fix any issues. Contributed by  Akira Ajisaka & Wangda Tan


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

Branch: refs/heads/HADOOP-13345
Commit: 43ebff2e354142bddcb42755766a965ae8a503a6
Parents: 1b5ccea
Author: Jian He <ji...@apache.org>
Authored: Mon Dec 5 11:39:34 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Mon Dec 5 11:39:34 2016 -0800

----------------------------------------------------------------------
 .../GetClusterNodeLabelsResponse.java           | 50 ++++++++++++++++----
 .../yarn/client/api/impl/YarnClientImpl.java    |  2 +-
 .../pb/GetClusterNodeLabelsResponsePBImpl.java  | 41 ++++++++++++++--
 .../yarn/security/ContainerTokenIdentifier.java | 25 ++++++++++
 .../state/InvalidStateTransitionException.java  | 22 ++-------
 .../state/InvalidStateTransitonException.java   | 19 ++++++--
 .../resourcemanager/TestClientRMService.java    |  4 +-
 7 files changed, 125 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43ebff2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeLabelsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeLabelsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeLabelsResponse.java
index cf6e683..cb2ccfb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeLabelsResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeLabelsResponse.java
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
@@ -28,18 +30,48 @@ import org.apache.hadoop.yarn.util.Records;
 @Public
 @Evolving
 public abstract class GetClusterNodeLabelsResponse {
+  /**
+   * Creates a new instance.
+   *
+   * @param labels Node labels
+   * @return response
+   * @deprecated Use {@link #newInstance(List)} instead.
+   */
+  @Deprecated
+  public static GetClusterNodeLabelsResponse newInstance(Set<String> labels) {
+    List<NodeLabel> list = new ArrayList<>();
+    for (String label : labels) {
+      list.add(NodeLabel.newInstance(label));
+    }
+    return newInstance(list);
+  }
+
   public static GetClusterNodeLabelsResponse newInstance(List<NodeLabel> labels) {
-    GetClusterNodeLabelsResponse request =
+    GetClusterNodeLabelsResponse response =
         Records.newRecord(GetClusterNodeLabelsResponse.class);
-    request.setNodeLabels(labels);
-    return request;
+    response.setNodeLabelList(labels);
+    return response;
   }
 
-  @Public
-  @Evolving
-  public abstract void setNodeLabels(List<NodeLabel> labels);
+  public abstract void setNodeLabelList(List<NodeLabel> labels);
+
+  public abstract List<NodeLabel> getNodeLabelList();
+
+  /**
+   * Set node labels to the response.
+   *
+   * @param labels Node labels
+   * @deprecated Use {@link #setNodeLabelList(List)} instead.
+   */
+  @Deprecated
+  public abstract void setNodeLabels(Set<String> labels);
 
-  @Public
-  @Evolving
-  public abstract List<NodeLabel> getNodeLabels();
+  /**
+   * Get node labels of the response.
+   *
+   * @return Node labels
+   * @deprecated Use {@link #getNodeLabelList()} instead.
+   */
+  @Deprecated
+  public abstract Set<String> getNodeLabels();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43ebff2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index a0f9678..50f1b490a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -899,7 +899,7 @@ public class YarnClientImpl extends YarnClient {
   @Override
   public List<NodeLabel> getClusterNodeLabels() throws YarnException, IOException {
     return rmClient.getClusterNodeLabels(
-        GetClusterNodeLabelsRequest.newInstance()).getNodeLabels();
+        GetClusterNodeLabelsRequest.newInstance()).getNodeLabelList();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43ebff2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeLabelsResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeLabelsResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeLabelsResponsePBImpl.java
index f569fb2..227abe9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeLabelsResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeLabelsResponsePBImpl.java
@@ -19,7 +19,9 @@
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
@@ -46,7 +48,7 @@ public class GetClusterNodeLabelsResponsePBImpl extends
     viaProto = true;
   }
 
-  public GetClusterNodeLabelsResponseProto getProto() {
+  public synchronized GetClusterNodeLabelsResponseProto getProto() {
     mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
@@ -101,14 +103,43 @@ public class GetClusterNodeLabelsResponsePBImpl extends
   }
 
   @Override
-  public void setNodeLabels(List<NodeLabel> updatedNodeLabels) {
+  public synchronized void setNodeLabelList(List<NodeLabel> nodeLabels) {
     maybeInitBuilder();
     this.updatedNodeLabels = new ArrayList<>();
-    if (updatedNodeLabels == null) {
+    if (nodeLabels == null) {
       builder.clearNodeLabels();
       return;
     }
-    this.updatedNodeLabels.addAll(updatedNodeLabels);
+    this.updatedNodeLabels.addAll(nodeLabels);
+  }
+
+  /**
+   * @deprecated Use {@link #getNodeLabelList()} instead.
+   */
+  @Override
+  @Deprecated
+  public synchronized Set<String> getNodeLabels() {
+    Set<String> set = new HashSet<>();
+    List<NodeLabel> labelList = getNodeLabelList();
+    if (labelList != null) {
+      for (NodeLabel label : labelList) {
+        set.add(label.getName());
+      }
+    }
+    return set;
+  }
+
+  /**
+   * @deprecated Use {@link #setNodeLabelList(List)} instead.
+   */
+  @Override
+  @Deprecated
+  public void setNodeLabels(Set<String> labels) {
+    List<NodeLabel> list = new ArrayList<>();
+    for (String s : labels) {
+      list.add(NodeLabel.newInstance(s));
+    }
+    setNodeLabelList(list);
   }
 
   private void initLocalNodeLabels() {
@@ -121,7 +152,7 @@ public class GetClusterNodeLabelsResponsePBImpl extends
   }
 
   @Override
-  public List<NodeLabel> getNodeLabels() {
+  public synchronized List<NodeLabel> getNodeLabelList() {
     if (this.updatedNodeLabels != null) {
       return this.updatedNodeLabels;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43ebff2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
index f8e9463..4b34998 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
@@ -73,6 +73,31 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
         CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
   }
 
+  /**
+   * Creates a instance.
+   *
+   * @param appSubmitter appSubmitter
+   * @param containerID container ID
+   * @param creationTime creation time
+   * @param expiryTimeStamp expiry timestamp
+   * @param hostName hostname
+   * @param logAggregationContext log aggregation context
+   * @param masterKeyId master key ID
+   * @param priority priority
+   * @param r resource needed by the container
+   * @param rmIdentifier ResourceManager identifier
+   * @deprecated Use one of the other constructors instead.
+   */
+  @Deprecated
+  public ContainerTokenIdentifier(ContainerId containerID, String hostName,
+      String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
+      long rmIdentifier, Priority priority, long creationTime,
+      LogAggregationContext logAggregationContext) {
+    this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
+        rmIdentifier, priority, creationTime, logAggregationContext,
+        CommonNodeLabelsManager.NO_LABEL);
+  }
+
   public ContainerTokenIdentifier(ContainerId containerID, String hostName,
       String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
       long rmIdentifier, Priority priority, long creationTime,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43ebff2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
index d10902a..51eafc9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.state;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 /**
  * The exception that happens when you call invalid state transition.
@@ -28,24 +27,13 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
  */
 @Public
 @Evolving
-public class InvalidStateTransitionException extends YarnRuntimeException {
+@SuppressWarnings("deprecation")
+public class InvalidStateTransitionException extends
+    InvalidStateTransitonException {
 
-  private static final long serialVersionUID = -6188669113571351684L;
-  private Enum<?> currentState;
-  private Enum<?> event;
+  private static final long serialVersionUID = 8610511635996283691L;
 
   public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
-    super("Invalid event: " + event + " at " + currentState);
-    this.currentState = currentState;
-    this.event = event;
+    super(currentState, event);
   }
-
-  public Enum<?> getCurrentState() {
-    return currentState;
-  }
-
-  public Enum<?> getEvent() {
-    return event;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43ebff2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
index eeb1b97..82f46cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
@@ -20,20 +20,31 @@ package org.apache.hadoop.yarn.state;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 /** @deprecated Use {@link InvalidStateTransitionException} instead. */
 
 @Public
 @Evolving
 @Deprecated
-public class InvalidStateTransitonException extends
-    InvalidStateTransitionException {
+public class InvalidStateTransitonException extends YarnRuntimeException {
 
-  private static final long serialVersionUID = 8610511635996283691L;
+  private static final long serialVersionUID = -6188669113571351684L;
+  private Enum<?> currentState;
+  private Enum<?> event;
 
   public InvalidStateTransitonException(Enum<?> currentState, Enum<?> event) {
-    super(currentState, event);
+    super("Invalid event: " + event + " at " + currentState);
+    this.currentState = currentState;
+    this.event = event;
   }
 
+  public Enum<?> getCurrentState() {
+    return currentState;
+  }
+
+  public Enum<?> getEvent() {
+    return event;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43ebff2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index f35efa0..12cdcf1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -1534,7 +1534,7 @@ public class TestClientRMService {
     // Get node labels collection
     GetClusterNodeLabelsResponse response = client
         .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance());
-    Assert.assertTrue(response.getNodeLabels().containsAll(
+    Assert.assertTrue(response.getNodeLabelList().containsAll(
         Arrays.asList(labelX, labelY)));
 
     // Get node labels mapping
@@ -1605,7 +1605,7 @@ public class TestClientRMService {
     // Get node labels collection
     GetClusterNodeLabelsResponse response = client
         .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance());
-    Assert.assertTrue(response.getNodeLabels().containsAll(
+    Assert.assertTrue(response.getNodeLabelList().containsAll(
         Arrays.asList(labelX, labelY, labelZ)));
 
     // Get labels to nodes mapping


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/50] [abbrv] hadoop git commit: Revert "HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang"

Posted by st...@apache.org.
Revert "HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang"

This reverts commit c7ff34f8dcca3a2024230c5383abd9299daa1b20.


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

Branch: refs/heads/HADOOP-13345
Commit: 08a7253bc0eb6c9155457feecb9c5cdc17c3a814
Parents: b2a3d6c
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Dec 5 23:08:49 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Dec 5 23:09:35 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  | 32 ------------
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 13 ++---
 .../hadoop/hdfs/web/resources/GetOpParam.java   | 12 +----
 .../web/resources/NamenodeWebHdfsMethods.java   | 17 -------
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    | 30 ------------
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 51 --------------------
 6 files changed, 4 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 12899f4..a75f4f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary.Builder;
 import org.apache.hadoop.fs.FileChecksum;
@@ -589,35 +588,4 @@ class JsonUtilClient {
         lastLocatedBlock, isLastBlockComplete, null, null);
   }
 
-  /** Convert a Json map to BlockLocation. **/
-  static BlockLocation toBlockLocation(Map<String, Object> m)
-      throws IOException{
-    long length = ((Number) m.get("length")).longValue();
-    long offset = ((Number) m.get("offset")).longValue();
-    boolean corrupt = Boolean.
-        getBoolean(m.get("corrupt").toString());
-    String[] storageIds = toStringArray(getList(m, "storageIds"));
-    String[] cachedHosts = toStringArray(getList(m, "cachedHosts"));
-    String[] hosts = toStringArray(getList(m, "hosts"));
-    String[] names = toStringArray(getList(m, "names"));
-    String[] topologyPaths = toStringArray(getList(m, "topologyPaths"));
-    StorageType[] storageTypes = toStorageTypeArray(
-        getList(m, "storageTypes"));
-    return new BlockLocation(names, hosts, cachedHosts,
-        topologyPaths, storageIds, storageTypes,
-        offset, length, corrupt);
-  }
-
-  static String[] toStringArray(List<?> list) {
-    if (list == null) {
-      return null;
-    } else {
-      final String[] array = new String[list.size()];
-      int i = 0;
-      for (Object object : list) {
-        array[i++] = object.toString();
-      }
-      return array;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index e82e9f6..23804b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -1610,20 +1610,13 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_FILE_BLOCK_LOCATIONS);
 
-    final HttpOpParam.Op op = GetOpParam.Op.GETFILEBLOCKLOCATIONS;
+    final HttpOpParam.Op op = GetOpParam.Op.GET_BLOCK_LOCATIONS;
     return new FsPathResponseRunner<BlockLocation[]>(op, p,
         new OffsetParam(offset), new LengthParam(length)) {
       @Override
-      @SuppressWarnings("unchecked")
       BlockLocation[] decodeResponse(Map<?,?> json) throws IOException {
-        List<?> list = JsonUtilClient.getList(json, "BlockLocations");
-        BlockLocation[] locations = new BlockLocation[list.size()];
-        for(int i=0; i<locations.length; i++) {
-          BlockLocation bl = JsonUtilClient.
-              toBlockLocation((Map<String, Object>) list.get(i));
-          locations[i] = bl;
-        }
-        return locations;
+        return DFSUtilClient.locatedBlocks2Locations(
+            JsonUtilClient.toLocatedBlocks(json));
       }
     }.run();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
index ccb0bb3..635e6d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
@@ -33,18 +33,8 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
 
-    /**
-     * GET_BLOCK_LOCATIONS is a private/stable API op. It returns a
-     * {@link org.apache.hadoop.hdfs.protocol.LocatedBlocks}
-     * json object.
-     */
+    /** GET_BLOCK_LOCATIONS is a private unstable op. */
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
-    /**
-     * GETFILEBLOCKLOCATIONS is the public op that complies with
-     * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations}
-     * interface.
-     */
-    GETFILEBLOCKLOCATIONS(false, HttpURLConnection.HTTP_OK),
     GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
     GETXATTRS(false, HttpURLConnection.HTTP_OK),
     GETTRASHROOT(false, HttpURLConnection.HTTP_OK),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 107d4ed..5d9b12a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -54,7 +54,6 @@ import javax.ws.rs.core.StreamingOutput;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -976,22 +975,6 @@ public class NamenodeWebHdfsMethods {
         return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       }
     }
-    case GETFILEBLOCKLOCATIONS:
-    {
-      final long offsetValue = offset.getValue();
-      final Long lengthValue = length.getValue();
-
-      try (final FileSystem fs = FileSystem.get(conf != null ?
-          conf : new Configuration())) {
-        BlockLocation[] locations = fs.getFileBlockLocations(
-            new org.apache.hadoop.fs.Path(fullpath),
-            offsetValue,
-            lengthValue != null? lengthValue: Long.MAX_VALUE);
-        final String js = JsonUtil.toJsonString("BlockLocations",
-            JsonUtil.toJsonArray(locations));
-        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
-      }
-    }
     case GET_BLOCK_LOCATIONS:
     {
       final long offsetValue = offset.getValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index affa861..a0dadbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -436,34 +436,4 @@ public class JsonUtil {
     return MAPPER.writeValueAsString(obj);
   }
 
-  public static Object[] toJsonArray(BlockLocation[] locations)
-      throws IOException {
-    if(locations == null) {
-      return null;
-    }
-    Object[] blockLocations = new Object[locations.length];
-    for(int i=0; i<locations.length; i++) {
-      blockLocations[i] = toJsonMap(locations[i]);
-    }
-    return blockLocations;
-  }
-
-  public static Map<String, Object> toJsonMap(
-      final BlockLocation blockLocation) throws IOException {
-    if (blockLocation == null) {
-      return null;
-    }
-
-    final Map<String, Object> m = new TreeMap<String, Object>();
-    m.put("length", blockLocation.getLength());
-    m.put("offset", blockLocation.getOffset());
-    m.put("corrupt", blockLocation.isCorrupt());
-    m.put("storageTypes", toJsonArray(blockLocation.getStorageTypes()));
-    m.put("storageIds", blockLocation.getStorageIds());
-    m.put("cachedHosts", blockLocation.getCachedHosts());
-    m.put("hosts", blockLocation.getHosts());
-    m.put("names", blockLocation.getNames());
-    m.put("topologyPaths", blockLocation.getTopologyPaths());
-    return m;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 82b708a..5386a45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -37,7 +37,6 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
-import java.util.Map;
 import java.util.Random;
 
 import org.apache.commons.io.IOUtils;
@@ -89,8 +88,6 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.type.MapType;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Mockito.doReturn;
@@ -855,46 +852,6 @@ public class TestWebHDFS {
         Assert.assertTrue(storageTypes != null && storageTypes.length > 0 &&
             storageTypes[0] == StorageType.DISK);
       }
-
-      // Query webhdfs REST API to get block locations
-      InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
-      URL url = new URL("http", addr.getHostString(), addr.getPort(),
-          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS");
-      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url);
-
-      String response = getResponse(url, "GET");
-      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response);
-      // Expected output from rest API
-      // { "BlockLoactions" : [{Block_Loation_Json}, ...] }
-      ObjectMapper mapper = new ObjectMapper();
-      MapType jsonType = mapper.getTypeFactory().constructMapType(
-          Map.class,
-          String.class,
-          BlockLocation[].class);
-      Map<String, BlockLocation[]> jsonMap = mapper.readValue(response,
-          jsonType);
-      BlockLocation[] array = jsonMap.get("BlockLocations");
-
-      for(int i=0; i<locations.length; i++) {
-        BlockLocation raw = locations[i];
-        BlockLocation rest = array[i];
-        Assert.assertEquals(raw.getLength(),
-            rest.getLength());
-        Assert.assertEquals(raw.getOffset(),
-            rest.getOffset());
-        Assert.assertArrayEquals(raw.getCachedHosts(),
-            rest.getCachedHosts());
-        Assert.assertArrayEquals(raw.getHosts(),
-            rest.getHosts());
-        Assert.assertArrayEquals(raw.getNames(),
-            rest.getNames());
-        Assert.assertArrayEquals(raw.getStorageIds(),
-            rest.getStorageIds());
-        Assert.assertArrayEquals(raw.getTopologyPaths(),
-            rest.getTopologyPaths());
-        Assert.assertArrayEquals(raw.getStorageTypes(),
-            rest.getStorageTypes());
-      }
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -902,14 +859,6 @@ public class TestWebHDFS {
     }
   }
 
-  private static String getResponse(URL url, String httpRequestType)
-      throws IOException {
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    conn.setRequestMethod(httpRequestType);
-    conn.setInstanceFollowRedirects(false);
-    return IOUtils.toString(conn.getInputStream());
-  }
-
   private WebHdfsFileSystem createWebHDFSAsTestUser(final Configuration conf,
       final URI uri, final String userName) throws Exception {
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/50] [abbrv] hadoop git commit: HADOOP-13847. KMSWebApp should close KeyProviderCryptoExtension. Contributed by John Zhuge.

Posted by st...@apache.org.
HADOOP-13847. KMSWebApp should close KeyProviderCryptoExtension. Contributed by John Zhuge.


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

Branch: refs/heads/HADOOP-13345
Commit: 291df5c7fb713d5442ee29eb3f272127afb05a3c
Parents: c51bfd2
Author: Xiao Chen <xi...@apache.org>
Authored: Mon Dec 5 09:34:39 2016 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Mon Dec 5 09:35:17 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/crypto/key/KeyProviderCryptoExtension.java  | 5 +++--
 .../org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java    | 7 ++++++-
 2 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/291df5c7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 1ecd9f6..0543222 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -427,8 +427,9 @@ public class KeyProviderCryptoExtension extends
 
   @Override
   public void close() throws IOException {
-    if (getKeyProvider() != null) {
-      getKeyProvider().close();
+    KeyProvider provider = getKeyProvider();
+    if (provider != null && provider != this) {
+      provider.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/291df5c7/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
index cd773dd..40ae19f 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
@@ -40,9 +40,9 @@ import javax.servlet.ServletContextEvent;
 import javax.servlet.ServletContextListener;
 
 import java.io.File;
+import java.io.IOException;
 import java.net.URI;
 import java.net.URL;
-import java.util.List;
 
 @InterfaceAudience.Private
 public class KMSWebApp implements ServletContextListener {
@@ -215,6 +215,11 @@ public class KMSWebApp implements ServletContextListener {
 
   @Override
   public void contextDestroyed(ServletContextEvent sce) {
+    try {
+      keyProviderCryptoExtension.close();
+    } catch (IOException ioe) {
+      LOG.error("Error closing KeyProviderCryptoExtension", ioe);
+    }
     kmsAudit.shutdown();
     kmsAcls.stopReloader();
     jmxReporter.stop();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[44/50] [abbrv] hadoop git commit: Revert "HDFS-11140. Directory Scanner should log startup message time correctly. Contributed by Yiqun Lin."

Posted by st...@apache.org.
Revert "HDFS-11140. Directory Scanner should log startup message time correctly. Contributed by Yiqun Lin."

This reverts commit 0857641f62778fad64e8158d78320efb0c8b417c.


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

Branch: refs/heads/HADOOP-13345
Commit: deb4743b2c3bb5c23842f0e224b89e41ef2d36d4
Parents: 0857641
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 8 13:37:06 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 8 13:37:06 2016 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/RawLocalFileSystem.java   |  9 ---------
 .../hadoop/hdfs/server/datanode/DirectoryScanner.java   | 12 ++++--------
 2 files changed, 4 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb4743b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index 84863a5..7bf429e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -791,15 +791,6 @@ public class RawLocalFileSystem extends FileSystem {
           pathToFile(p).toPath(), BasicFileAttributeView.class);
       FileTime fmtime = (mtime >= 0) ? FileTime.fromMillis(mtime) : null;
       FileTime fatime = (atime >= 0) ? FileTime.fromMillis(atime) : null;
-
-      // On some macOS environment, BasicFileAttributeView.setTimes
-      // does not set times correctly when the argument of accessTime is null.
-      // TODO: Remove this after the issue is fixed.
-      if (fatime == null && Shell.MAC) {
-        FileStatus f = getFileStatus(p);
-        fatime = FileTime.fromMillis(f.getAccessTime());
-      }
-
       view.setTimes(fmtime, fatime, null);
     } catch (NoSuchFileException e) {
       throw new FileNotFoundException("File " + p + " does not exist");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb4743b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 18188dd..e2baf32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -37,8 +37,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -63,7 +61,7 @@ public class DirectoryScanner implements Runnable {
   private static final int MILLIS_PER_SECOND = 1000;
   private static final String START_MESSAGE =
       "Periodic Directory Tree Verification scan"
-      + " starting at %s with interval of %dms";
+      + " starting at %dms with interval of %dms";
   private static final String START_MESSAGE_WITH_THROTTLE = START_MESSAGE
       + " and throttle limit of %dms/s";
 
@@ -270,12 +268,10 @@ public class DirectoryScanner implements Runnable {
     String logMsg;
 
     if (throttleLimitMsPerSec < MILLIS_PER_SECOND) {
-      logMsg = String.format(START_MESSAGE_WITH_THROTTLE,
-          FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs,
-          throttleLimitMsPerSec);
+      logMsg = String.format(START_MESSAGE_WITH_THROTTLE, firstScanTime,
+          scanPeriodMsecs, throttleLimitMsPerSec);
     } else {
-      logMsg = String.format(START_MESSAGE,
-          FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs);
+      logMsg = String.format(START_MESSAGE, firstScanTime, scanPeriodMsecs);
     }
 
     LOG.info(logMsg);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[46/50] [abbrv] hadoop git commit: YARN-5970. Validate application update timeout request parameters. Contributed by Rohith Sharma K S.

Posted by st...@apache.org.
YARN-5970. Validate application update timeout request parameters. Contributed by Rohith Sharma K S.


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

Branch: refs/heads/HADOOP-13345
Commit: 74d0066d3392169bec872f438a0818e2f5323010
Parents: 9ef89ed
Author: Sunil G <su...@apache.org>
Authored: Thu Dec 8 15:53:56 2016 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Dec 8 15:53:56 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/util/Times.java |  3 ++
 .../server/resourcemanager/RMServerUtils.java   |  3 +-
 .../resourcemanager/webapp/RMWebServices.java   | 10 ++---
 .../TestRMWebServicesAppsModification.java      | 39 ++++++++++++++------
 4 files changed, 38 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/74d0066d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Times.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Times.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Times.java
index f113bd3..3c41558 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Times.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Times.java
@@ -105,6 +105,9 @@ public class Times {
    */
   public static long parseISO8601ToLocalTimeInMillis(String isoString)
       throws ParseException {
+    if (isoString == null) {
+      throw new ParseException("Invalid input.", -1);
+    }
     return isoFormat.get().parse(isoString).getTime();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74d0066d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index a0cdf68..74898ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -507,7 +507,8 @@ public class RMServerUtils {
         } catch (ParseException ex) {
           String message =
               "Expire time is not in ISO8601 format. ISO8601 supported "
-                  + "format is yyyy-MM-dd'T'HH:mm:ss.SSSZ";
+                  + "format is yyyy-MM-dd'T'HH:mm:ss.SSSZ. Configured "
+                  + "timeout value is " + timeout.getValue();
           throw new YarnException(message, ex);
         }
         if (expireTime < currentTimeMillis) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74d0066d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index a46fb81..bd0602b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -2434,7 +2434,7 @@ public class RMWebServices extends WebServices {
   }
 
   @GET
-  @Path("/apps/{appid}/timeout/{type}")
+  @Path("/apps/{appid}/timeouts/{type}")
   @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
       MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   public AppTimeoutInfo getAppTimeout(@Context HttpServletRequest hsr,
@@ -2571,10 +2571,10 @@ public class RMWebServices extends WebServices {
   private Response updateApplicationTimeouts(final RMApp app,
       UserGroupInformation callerUGI, final AppTimeoutInfo appTimeout)
       throws IOException, InterruptedException {
-
-    if (appTimeout.getTimeoutType() == null) {
-      return Response.status(Status.BAD_REQUEST).entity("Timeout type is null.")
-          .build();
+    if (appTimeout.getTimeoutType() == null
+        || appTimeout.getExpireTime() == null) {
+      return Response.status(Status.BAD_REQUEST)
+          .entity("Timeout type or ExpiryTime is null.").build();
     }
 
     String userName = callerUGI.getUserName();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74d0066d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
index 6458471..5d99e0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
@@ -1328,19 +1328,11 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
               ApplicationTimeoutType.LIFETIME, "UNLIMITED", -1);
         }
 
-        AppTimeoutInfo timeoutUpdate = new AppTimeoutInfo();
         long timeOutFromNow = 60;
         String expireTime = Times
             .formatISO8601(System.currentTimeMillis() + timeOutFromNow * 1000);
-        timeoutUpdate.setTimeoutType(ApplicationTimeoutType.LIFETIME);
-        timeoutUpdate.setExpiryTime(expireTime);
-
-        Object entity;
-        if (contentType.equals(MediaType.APPLICATION_JSON_TYPE)) {
-          entity = appTimeoutToJSON(timeoutUpdate);
-        } else {
-          entity = timeoutUpdate;
-        }
+        Object entity = getAppTimeoutInfoEntity(ApplicationTimeoutType.LIFETIME,
+            contentType, expireTime);
         response = this
             .constructWebResource("apps", app.getApplicationId().toString(),
                 "timeout")
@@ -1361,10 +1353,20 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
               expireTime, timeOutFromNow);
         }
 
+        // verify for negative cases
+        entity = getAppTimeoutInfoEntity(null,
+            contentType, null);
+        response = this
+            .constructWebResource("apps", app.getApplicationId().toString(),
+                "timeout")
+            .entity(entity, contentType).accept(mediaType)
+            .put(ClientResponse.class);
+        assertResponseStatusCode(Status.BAD_REQUEST, response.getStatusInfo());
+
         // invoke get
         response =
             this.constructWebResource("apps", app.getApplicationId().toString(),
-                "timeout", ApplicationTimeoutType.LIFETIME.toString())
+                "timeouts", ApplicationTimeoutType.LIFETIME.toString())
                 .accept(mediaType).get(ClientResponse.class);
         assertResponseStatusCode(Status.OK, response.getStatusInfo());
         if (mediaType.contains(MediaType.APPLICATION_JSON)) {
@@ -1376,6 +1378,21 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     rm.stop();
   }
 
+  private Object getAppTimeoutInfoEntity(ApplicationTimeoutType type,
+      MediaType contentType, String expireTime) throws Exception {
+    AppTimeoutInfo timeoutUpdate = new AppTimeoutInfo();
+    timeoutUpdate.setTimeoutType(type);
+    timeoutUpdate.setExpiryTime(expireTime);
+
+    Object entity;
+    if (contentType.equals(MediaType.APPLICATION_JSON_TYPE)) {
+      entity = appTimeoutToJSON(timeoutUpdate);
+    } else {
+      entity = timeoutUpdate;
+    }
+    return entity;
+  }
+
   protected static void verifyAppTimeoutJson(ClientResponse response,
       ApplicationTimeoutType type, String expireTime, long timeOutFromNow)
       throws JSONException {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[41/50] [abbrv] hadoop git commit: HADOOP-13842. Update jackson from 1.9.13 to 2.x in hadoop-maven-plugins.

Posted by st...@apache.org.
HADOOP-13842. Update jackson from 1.9.13 to 2.x in hadoop-maven-plugins.


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

Branch: refs/heads/HADOOP-13345
Commit: 3c774b8c98540642eeb706065cf0275f9891d3b2
Parents: ea2895f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 8 11:06:25 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 8 11:06:25 2016 +0900

----------------------------------------------------------------------
 hadoop-maven-plugins/pom.xml                            | 12 ++++--------
 .../apache/hadoop/maven/plugin/protoc/ProtocRunner.java |  5 +++--
 2 files changed, 7 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c774b8c/hadoop-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/pom.xml b/hadoop-maven-plugins/pom.xml
index 51b8f91..eae03a1 100644
--- a/hadoop-maven-plugins/pom.xml
+++ b/hadoop-maven-plugins/pom.xml
@@ -47,17 +47,13 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-core-asl</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
   <build>
     <plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c774b8c/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocRunner.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocRunner.java
index ef15884..e83a8cd 100644
--- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocRunner.java
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocRunner.java
@@ -13,14 +13,15 @@
  */
 package org.apache.hadoop.maven.plugin.protoc;
 
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 import org.apache.hadoop.maven.plugin.util.Exec;
 import org.apache.hadoop.maven.plugin.util.FileSetUtils;
 import org.apache.maven.model.FileSet;
 import org.apache.maven.plugin.AbstractMojo;
 import org.apache.maven.plugin.MojoExecutionException;
 import org.apache.maven.project.MavenProject;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.type.TypeReference;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/50] [abbrv] hadoop git commit: YARN-5746. The state of the parentQueue and its childQueues should be synchronized. Contributed by Xuan Gong

Posted by st...@apache.org.
YARN-5746. The state of the parentQueue and its childQueues should be synchronized. Contributed by Xuan Gong


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

Branch: refs/heads/HADOOP-13345
Commit: f885160f4ac56a0999e3b051eb7bccce928c1c33
Parents: 4113ec5
Author: Jian He <ji...@apache.org>
Authored: Fri Dec 2 16:17:31 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Fri Dec 2 16:17:31 2016 -0800

----------------------------------------------------------------------
 .../scheduler/capacity/AbstractCSQueue.java     | 26 +++++-
 .../CapacitySchedulerConfiguration.java         | 22 ++++-
 .../scheduler/capacity/TestQueueState.java      | 96 ++++++++++++++++++++
 3 files changed, 139 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f885160f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index 3daabaf..dd2f0d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -291,7 +291,8 @@ public abstract class AbstractCSQueue implements CSQueue {
 
       authorizer = YarnAuthorizationProvider.getInstance(csContext.getConf());
 
-      this.state = csContext.getConfiguration().getState(getQueuePath());
+      initializeQueueState();
+
       this.acls = csContext.getConfiguration().getAcls(getQueuePath());
 
       // Update metrics
@@ -330,6 +331,29 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
   }
 
+  private void initializeQueueState() {
+    // inherit from parent if state not set, only do this when we are not root
+    if (parent != null) {
+      QueueState configuredState = csContext.getConfiguration()
+          .getConfiguredState(getQueuePath());
+      QueueState parentState = parent.getState();
+      if (configuredState == null) {
+        this.state = parentState;
+      } else if (configuredState == QueueState.RUNNING
+          && parentState == QueueState.STOPPED) {
+        throw new IllegalArgumentException(
+            "The parent queue:" + parent.getQueueName() + " state is STOPPED, "
+            + "child queue:" + queueName + " state cannot be RUNNING.");
+      } else {
+        this.state = configuredState;
+      }
+    } else {
+      // if this is the root queue, get the state from the configuration.
+      // if the state is not set, use RUNNING as default state.
+      this.state = csContext.getConfiguration().getState(getQueuePath());
+    }
+  }
+
   protected QueueInfo getQueueInfo() {
     // Deliberately doesn't use lock here, because this method will be invoked
     // from schedulerApplicationAttempt, to avoid deadlock, sacrifice

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f885160f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index f8335a8..bfaeba4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -448,12 +448,26 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     setFloat(getQueuePrefix(queue) + USER_LIMIT_FACTOR, userLimitFactor); 
   }
   
-  public QueueState getState(String queue) {
+  public QueueState getConfiguredState(String queue) {
     String state = get(getQueuePrefix(queue) + STATE);
-    return (state != null) ? 
-        QueueState.valueOf(StringUtils.toUpperCase(state)) : QueueState.RUNNING;
+    if (state == null) {
+      return null;
+    } else {
+      return QueueState.valueOf(StringUtils.toUpperCase(state));
+    }
   }
-  
+
+  public QueueState getState(String queue) {
+    QueueState state = getConfiguredState(queue);
+    return (state == null) ? QueueState.RUNNING : state;
+  }
+
+  @Private
+  @VisibleForTesting
+  public void setState(String queue, QueueState state) {
+    set(getQueuePrefix(queue) + STATE, state.name());
+  }
+
   public void setAccessibleNodeLabels(String queue, Set<String> labels) {
     if (labels == null) {
       return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f885160f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueState.java
new file mode 100644
index 0000000..bd878b7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueState.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.io.IOException;
+import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test Queue States.
+ */
+public class TestQueueState {
+
+  private static final String Q1 = "q1";
+  private static final String Q2 = "q2";
+
+  private final static String Q1_PATH =
+      CapacitySchedulerConfiguration.ROOT + "." + Q1;
+  private final static String Q2_PATH =
+      Q1_PATH + "." + Q2;
+  private CapacityScheduler cs;
+  private YarnConfiguration conf;
+
+  @Test (timeout = 15000)
+  public void testQueueState() throws IOException {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {Q1});
+    csConf.setQueues(Q1_PATH, new String[] {Q2});
+
+    csConf.setCapacity(Q1_PATH, 100);
+    csConf.setCapacity(Q2_PATH, 100);
+
+    conf = new YarnConfiguration(csConf);
+    cs = new CapacityScheduler();
+
+    RMContext rmContext = TestUtils.getMockRMContext();
+    cs.setConf(conf);
+    cs.setRMContext(rmContext);
+    cs.init(conf);
+
+    //by default, the state of both queues should be RUNNING
+    Assert.assertEquals(QueueState.RUNNING, cs.getQueue(Q1).getState());
+    Assert.assertEquals(QueueState.RUNNING, cs.getQueue(Q2).getState());
+
+    // Change the state of Q1 to STOPPED, and re-initiate the CS
+    csConf.setState(Q1_PATH, QueueState.STOPPED);
+    conf = new YarnConfiguration(csConf);
+    cs.reinitialize(conf, rmContext);
+    // The state of Q1 and its child: Q2 should be STOPPED
+    Assert.assertEquals(QueueState.STOPPED, cs.getQueue(Q1).getState());
+    Assert.assertEquals(QueueState.STOPPED, cs.getQueue(Q2).getState());
+
+    // Change the state of Q1 to RUNNING, and change the state of Q2 to STOPPED
+    csConf.setState(Q1_PATH, QueueState.RUNNING);
+    csConf.setState(Q2_PATH, QueueState.STOPPED);
+    conf = new YarnConfiguration(csConf);
+    // reinitialize the CS, the operation should be successful
+    cs.reinitialize(conf, rmContext);
+    Assert.assertEquals(QueueState.RUNNING, cs.getQueue(Q1).getState());
+    Assert.assertEquals(QueueState.STOPPED, cs.getQueue(Q2).getState());
+
+    // Change the state of Q1 to STOPPED, and change the state of Q2 to RUNNING
+    csConf.setState(Q1_PATH, QueueState.STOPPED);
+    csConf.setState(Q2_PATH, QueueState.RUNNING);
+    conf = new YarnConfiguration(csConf);
+    // reinitialize the CS, the operation should be failed.
+    try {
+      cs.reinitialize(conf, rmContext);
+      Assert.fail("Should throw an Exception.");
+    } catch (Exception ex) {
+      Assert.assertTrue(ex.getCause().getMessage().contains(
+          "The parent queue:q1 state is STOPPED, "
+          + "child queue:q2 state cannot be RUNNING."));
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[42/50] [abbrv] hadoop git commit: MAPREDUCE-4683. Create and distribute hadoop-mapreduce-client-core-tests.jar.

Posted by st...@apache.org.
MAPREDUCE-4683. Create and distribute hadoop-mapreduce-client-core-tests.jar.


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

Branch: refs/heads/HADOOP-13345
Commit: f54afdb83b2567f63f0dc94f5f4e26ead2eefc5e
Parents: 3c774b8
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 8 11:27:58 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 8 11:27:58 2016 +0900

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-core/pom.xml            | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f54afdb8/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index 38a7725..aefba7f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -61,6 +61,22 @@
   <build>
     <plugins>
       <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <configuration>
+              <excludes>
+                <exclude>**/mapred-queues.xml</exclude>
+              </excludes>
+            </configuration>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
 	<configuration>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[38/50] [abbrv] hadoop git commit: MAPREDUCE-6817. The format of job start time in JHS is different from those of submit and finish time. (Haibo Chen via kasha)

Posted by st...@apache.org.
MAPREDUCE-6817. The format of job start time in JHS is different from those of submit and finish time. (Haibo Chen via kasha)


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

Branch: refs/heads/HADOOP-13345
Commit: a793cec2c9a7ad80aaf67be9dec2245e8ecce63c
Parents: 9f5d2c4
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Wed Dec 7 13:36:49 2016 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Wed Dec 7 13:36:49 2016 -0800

----------------------------------------------------------------------
 .../mapreduce/v2/hs/webapp/HsJobsBlock.java     |  2 +-
 .../mapreduce/v2/hs/webapp/dao/JobInfo.java     | 15 +++++++++-
 .../mapreduce/v2/hs/webapp/dao/TestJobInfo.java | 29 +++++++++++++++++++-
 3 files changed, 43 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a793cec2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
index 92c5e2b..b234ca3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
@@ -79,7 +79,7 @@ public class HsJobsBlock extends HtmlBlock {
       JobInfo job = new JobInfo(j);
       jobsTableData.append("[\"")
       .append(dateFormat.format(new Date(job.getSubmitTime()))).append("\",\"")
-      .append(job.getStartTimeStr()).append("\",\"")
+      .append(job.getFormattedStartTimeStr(dateFormat)).append("\",\"")
       .append(dateFormat.format(new Date(job.getFinishTime()))).append("\",\"")
       .append("<a href='").append(url("job", job.getId())).append("'>")
       .append(job.getId()).append("</a>\",\"")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a793cec2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java
index eddbccf..d12a729 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.mapreduce.v2.hs.webapp.dao;
 
+import java.text.DateFormat;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
@@ -27,6 +28,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@@ -44,7 +46,8 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 @XmlRootElement(name = "job")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class JobInfo {
-  private static final String NA = "N/A";
+  @VisibleForTesting
+  static final String NA = "N/A";
 
   protected long submitTime;
   protected long startTime;
@@ -228,6 +231,16 @@ public class JobInfo {
     return this.startTime;
   }
 
+  public String getFormattedStartTimeStr(final DateFormat dateFormat) {
+    String str = NA;
+
+    if (startTime >= 0) {
+      str = dateFormat.format(new Date(startTime));
+    }
+
+    return str;
+  }
+
   public String getStartTimeStr() {
     String str = NA;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a793cec2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/TestJobInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/TestJobInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/TestJobInfo.java
index 51f87e0..c8b749f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/TestJobInfo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/TestJobInfo.java
@@ -22,6 +22,8 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.HashMap;
 
@@ -153,7 +155,7 @@ public class TestJobInfo {
     when(job.getID()).thenReturn(jobId);
 
     JobInfo jobInfo = new JobInfo(job);
-    Assert.assertEquals("N/A", jobInfo.getStartTimeStr());
+    Assert.assertEquals(JobInfo.NA, jobInfo.getStartTimeStr());
 
     Date date = new Date();
     when(jobReport.getStartTime()).thenReturn(date.getTime());
@@ -161,4 +163,29 @@ public class TestJobInfo {
     jobInfo = new JobInfo(job);
     Assert.assertEquals(date.toString(), jobInfo.getStartTimeStr());
   }
+
+  @Test
+  public void testGetFormattedStartTimeStr() {
+    JobReport jobReport = mock(JobReport.class);
+    when(jobReport.getStartTime()).thenReturn(-1L);
+
+    Job job = mock(Job.class);
+    when(job.getReport()).thenReturn(jobReport);
+    when(job.getName()).thenReturn("TestJobInfo");
+    when(job.getState()).thenReturn(JobState.SUCCEEDED);
+
+    JobId jobId = MRBuilderUtils.newJobId(1L, 1, 1);
+    when(job.getID()).thenReturn(jobId);
+    DateFormat dateFormat = new SimpleDateFormat();
+
+    JobInfo jobInfo = new JobInfo(job);
+    Assert.assertEquals(
+        JobInfo.NA, jobInfo.getFormattedStartTimeStr(dateFormat));
+
+    Date date = new Date();
+    when(jobReport.getStartTime()).thenReturn(date.getTime());
+    jobInfo = new JobInfo(job);
+    Assert.assertEquals(
+        dateFormat.format(date), jobInfo.getFormattedStartTimeStr(dateFormat));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/50] [abbrv] hadoop git commit: HADOOP-13835. Move Google Test Framework code from mapreduce to hadoop-common. Contributed by Varun Vasudev.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/gtest-all.cc
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/gtest-all.cc b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/gtest-all.cc
deleted file mode 100644
index 4f8c08a..0000000
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/gtest-all.cc
+++ /dev/null
@@ -1,10403 +0,0 @@
-// Copyright 2008, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Author: mheule@google.com (Markus Heule)
-//
-// Google C++ Testing Framework (Google Test)
-//
-// Sometimes it's desirable to build Google Test by compiling a single file.
-// This file serves this purpose.
-
-// This line ensures that gtest.h can be compiled on its own, even
-// when it's fused.
-#include "gtest/gtest.h"
-
-// The following lines pull in the real gtest *.cc files.
-// Copyright 2005, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Author: wan@google.com (Zhanyong Wan)
-//
-// The Google C++ Testing Framework (Google Test)
-
-// Copyright 2007, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Author: wan@google.com (Zhanyong Wan)
-//
-// Utilities for testing Google Test itself and code that uses Google Test
-// (e.g. frameworks built on top of Google Test).
-
-#ifndef GTEST_INCLUDE_GTEST_GTEST_SPI_H_
-#define GTEST_INCLUDE_GTEST_GTEST_SPI_H_
-
-
-namespace testing {
-
-// This helper class can be used to mock out Google Test failure reporting
-// so that we can test Google Test or code that builds on Google Test.
-//
-// An object of this class appends a TestPartResult object to the
-// TestPartResultArray object given in the constructor whenever a Google Test
-// failure is reported. It can either intercept only failures that are
-// generated in the same thread that created this object or it can intercept
-// all generated failures. The scope of this mock object can be controlled with
-// the second argument to the two arguments constructor.
-class GTEST_API_ ScopedFakeTestPartResultReporter
-    : public TestPartResultReporterInterface {
- public:
-  // The two possible mocking modes of this object.
-  enum InterceptMode {
-    INTERCEPT_ONLY_CURRENT_THREAD,  // Intercepts only thread local failures.
-    INTERCEPT_ALL_THREADS           // Intercepts all failures.
-  };
-
-  // The c'tor sets this object as the test part result reporter used
-  // by Google Test.  The 'result' parameter specifies where to report the
-  // results. This reporter will only catch failures generated in the current
-  // thread. DEPRECATED
-  explicit ScopedFakeTestPartResultReporter(TestPartResultArray* result);
-
-  // Same as above, but you can choose the interception scope of this object.
-  ScopedFakeTestPartResultReporter(InterceptMode intercept_mode,
-                                   TestPartResultArray* result);
-
-  // The d'tor restores the previous test part result reporter.
-  virtual ~ScopedFakeTestPartResultReporter();
-
-  // Appends the TestPartResult object to the TestPartResultArray
-  // received in the constructor.
-  //
-  // This method is from the TestPartResultReporterInterface
-  // interface.
-  virtual void ReportTestPartResult(const TestPartResult& result);
- private:
-  void Init();
-
-  const InterceptMode intercept_mode_;
-  TestPartResultReporterInterface* old_reporter_;
-  TestPartResultArray* const result_;
-
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(ScopedFakeTestPartResultReporter);
-};
-
-namespace internal {
-
-// A helper class for implementing EXPECT_FATAL_FAILURE() and
-// EXPECT_NONFATAL_FAILURE().  Its destructor verifies that the given
-// TestPartResultArray contains exactly one failure that has the given
-// type and contains the given substring.  If that's not the case, a
-// non-fatal failure will be generated.
-class GTEST_API_ SingleFailureChecker {
- public:
-  // The constructor remembers the arguments.
-  SingleFailureChecker(const TestPartResultArray* results,
-                       TestPartResult::Type type,
-                       const string& substr);
-  ~SingleFailureChecker();
- private:
-  const TestPartResultArray* const results_;
-  const TestPartResult::Type type_;
-  const string substr_;
-
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(SingleFailureChecker);
-};
-
-}  // namespace internal
-
-}  // namespace testing
-
-// A set of macros for testing Google Test assertions or code that's expected
-// to generate Google Test fatal failures.  It verifies that the given
-// statement will cause exactly one fatal Google Test failure with 'substr'
-// being part of the failure message.
-//
-// There are two different versions of this macro. EXPECT_FATAL_FAILURE only
-// affects and considers failures generated in the current thread and
-// EXPECT_FATAL_FAILURE_ON_ALL_THREADS does the same but for all threads.
-//
-// The verification of the assertion is done correctly even when the statement
-// throws an exception or aborts the current function.
-//
-// Known restrictions:
-//   - 'statement' cannot reference local non-static variables or
-//     non-static members of the current object.
-//   - 'statement' cannot return a value.
-//   - You cannot stream a failure message to this macro.
-//
-// Note that even though the implementations of the following two
-// macros are much alike, we cannot refactor them to use a common
-// helper macro, due to some peculiarity in how the preprocessor
-// works.  The AcceptsMacroThatExpandsToUnprotectedComma test in
-// gtest_unittest.cc will fail to compile if we do that.
-#define EXPECT_FATAL_FAILURE(statement, substr) \
-  do { \
-    class GTestExpectFatalFailureHelper {\
-     public:\
-      static void Execute() { statement; }\
-    };\
-    ::testing::TestPartResultArray gtest_failures;\
-    ::testing::internal::SingleFailureChecker gtest_checker(\
-        &gtest_failures, ::testing::TestPartResult::kFatalFailure, (substr));\
-    {\
-      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
-          ::testing::ScopedFakeTestPartResultReporter:: \
-          INTERCEPT_ONLY_CURRENT_THREAD, &gtest_failures);\
-      GTestExpectFatalFailureHelper::Execute();\
-    }\
-  } while (::testing::internal::AlwaysFalse())
-
-#define EXPECT_FATAL_FAILURE_ON_ALL_THREADS(statement, substr) \
-  do { \
-    class GTestExpectFatalFailureHelper {\
-     public:\
-      static void Execute() { statement; }\
-    };\
-    ::testing::TestPartResultArray gtest_failures;\
-    ::testing::internal::SingleFailureChecker gtest_checker(\
-        &gtest_failures, ::testing::TestPartResult::kFatalFailure, (substr));\
-    {\
-      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
-          ::testing::ScopedFakeTestPartResultReporter:: \
-          INTERCEPT_ALL_THREADS, &gtest_failures);\
-      GTestExpectFatalFailureHelper::Execute();\
-    }\
-  } while (::testing::internal::AlwaysFalse())
-
-// A macro for testing Google Test assertions or code that's expected to
-// generate Google Test non-fatal failures.  It asserts that the given
-// statement will cause exactly one non-fatal Google Test failure with 'substr'
-// being part of the failure message.
-//
-// There are two different versions of this macro. EXPECT_NONFATAL_FAILURE only
-// affects and considers failures generated in the current thread and
-// EXPECT_NONFATAL_FAILURE_ON_ALL_THREADS does the same but for all threads.
-//
-// 'statement' is allowed to reference local variables and members of
-// the current object.
-//
-// The verification of the assertion is done correctly even when the statement
-// throws an exception or aborts the current function.
-//
-// Known restrictions:
-//   - You cannot stream a failure message to this macro.
-//
-// Note that even though the implementations of the following two
-// macros are much alike, we cannot refactor them to use a common
-// helper macro, due to some peculiarity in how the preprocessor
-// works.  If we do that, the code won't compile when the user gives
-// EXPECT_NONFATAL_FAILURE() a statement that contains a macro that
-// expands to code containing an unprotected comma.  The
-// AcceptsMacroThatExpandsToUnprotectedComma test in gtest_unittest.cc
-// catches that.
-//
-// For the same reason, we have to write
-//   if (::testing::internal::AlwaysTrue()) { statement; }
-// instead of
-//   GTEST_SUPPRESS_UNREACHABLE_CODE_WARNING_BELOW_(statement)
-// to avoid an MSVC warning on unreachable code.
-#define EXPECT_NONFATAL_FAILURE(statement, substr) \
-  do {\
-    ::testing::TestPartResultArray gtest_failures;\
-    ::testing::internal::SingleFailureChecker gtest_checker(\
-        &gtest_failures, ::testing::TestPartResult::kNonFatalFailure, \
-        (substr));\
-    {\
-      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
-          ::testing::ScopedFakeTestPartResultReporter:: \
-          INTERCEPT_ONLY_CURRENT_THREAD, &gtest_failures);\
-      if (::testing::internal::AlwaysTrue()) { statement; }\
-    }\
-  } while (::testing::internal::AlwaysFalse())
-
-#define EXPECT_NONFATAL_FAILURE_ON_ALL_THREADS(statement, substr) \
-  do {\
-    ::testing::TestPartResultArray gtest_failures;\
-    ::testing::internal::SingleFailureChecker gtest_checker(\
-        &gtest_failures, ::testing::TestPartResult::kNonFatalFailure, \
-        (substr));\
-    {\
-      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
-          ::testing::ScopedFakeTestPartResultReporter::INTERCEPT_ALL_THREADS, \
-          &gtest_failures);\
-      if (::testing::internal::AlwaysTrue()) { statement; }\
-    }\
-  } while (::testing::internal::AlwaysFalse())
-
-#endif  // GTEST_INCLUDE_GTEST_GTEST_SPI_H_
-
-#include <ctype.h>
-#include <math.h>
-#include <stdarg.h>
-#include <stdio.h>
-#include <stdlib.h>
-#include <time.h>
-#include <wchar.h>
-#include <wctype.h>
-
-#include <algorithm>
-#include <iomanip>
-#include <limits>
-#include <list>
-#include <map>
-#include <ostream>  // NOLINT
-#include <sstream>
-#include <vector>
-
-#if GTEST_OS_LINUX
-
-// TODO(kenton@google.com): Use autoconf to detect availability of
-// gettimeofday().
-# define GTEST_HAS_GETTIMEOFDAY_ 1
-
-# include <fcntl.h>  // NOLINT
-# include <limits.h>  // NOLINT
-# include <sched.h>  // NOLINT
-// Declares vsnprintf().  This header is not available on Windows.
-# include <strings.h>  // NOLINT
-# include <sys/mman.h>  // NOLINT
-# include <sys/time.h>  // NOLINT
-# include <unistd.h>  // NOLINT
-# include <string>
-
-#elif GTEST_OS_SYMBIAN
-# define GTEST_HAS_GETTIMEOFDAY_ 1
-# include <sys/time.h>  // NOLINT
-
-#elif GTEST_OS_ZOS
-# define GTEST_HAS_GETTIMEOFDAY_ 1
-# include <sys/time.h>  // NOLINT
-
-// On z/OS we additionally need strings.h for strcasecmp.
-# include <strings.h>  // NOLINT
-
-#elif GTEST_OS_WINDOWS_MOBILE  // We are on Windows CE.
-
-# include <windows.h>  // NOLINT
-# undef min
-
-#elif GTEST_OS_WINDOWS  // We are on Windows proper.
-
-# include <io.h>  // NOLINT
-# include <sys/timeb.h>  // NOLINT
-# include <sys/types.h>  // NOLINT
-# include <sys/stat.h>  // NOLINT
-
-# if GTEST_OS_WINDOWS_MINGW
-// MinGW has gettimeofday() but not _ftime64().
-// TODO(kenton@google.com): Use autoconf to detect availability of
-//   gettimeofday().
-// TODO(kenton@google.com): There are other ways to get the time on
-//   Windows, like GetTickCount() or GetSystemTimeAsFileTime().  MinGW
-//   supports these.  consider using them instead.
-#  define GTEST_HAS_GETTIMEOFDAY_ 1
-#  include <sys/time.h>  // NOLINT
-# endif  // GTEST_OS_WINDOWS_MINGW
-
-// cpplint thinks that the header is already included, so we want to
-// silence it.
-# include <windows.h>  // NOLINT
-# undef min
-
-#else
-
-// Assume other platforms have gettimeofday().
-// TODO(kenton@google.com): Use autoconf to detect availability of
-//   gettimeofday().
-# define GTEST_HAS_GETTIMEOFDAY_ 1
-
-// cpplint thinks that the header is already included, so we want to
-// silence it.
-# include <sys/time.h>  // NOLINT
-# include <unistd.h>  // NOLINT
-
-#endif  // GTEST_OS_LINUX
-
-#if GTEST_HAS_EXCEPTIONS
-# include <stdexcept>
-#endif
-
-#if GTEST_CAN_STREAM_RESULTS_
-# include <arpa/inet.h>  // NOLINT
-# include <netdb.h>  // NOLINT
-# include <sys/socket.h>  // NOLINT
-# include <sys/types.h>  // NOLINT
-#endif
-
-// Indicates that this translation unit is part of Google Test's
-// implementation.  It must come before gtest-internal-inl.h is
-// included, or there will be a compiler error.  This trick is to
-// prevent a user from accidentally including gtest-internal-inl.h in
-// his code.
-#define GTEST_IMPLEMENTATION_ 1
-// Copyright 2005, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-// Utility functions and classes used by the Google C++ testing framework.
-//
-// Author: wan@google.com (Zhanyong Wan)
-//
-// This file contains purely Google Test's internal implementation.  Please
-// DO NOT #INCLUDE IT IN A USER PROGRAM.
-
-#ifndef GTEST_SRC_GTEST_INTERNAL_INL_H_
-#define GTEST_SRC_GTEST_INTERNAL_INL_H_
-
-// GTEST_IMPLEMENTATION_ is defined to 1 iff the current translation unit is
-// part of Google Test's implementation; otherwise it's undefined.
-#if !GTEST_IMPLEMENTATION_
-// If this file is included from the user's code, just say no.
-# error "gtest-internal-inl.h is part of Google Test's internal implementation."
-# error "It must not be included except by Google Test itself."
-#endif  // GTEST_IMPLEMENTATION_
-
-#ifndef _WIN32_WCE
-# include <errno.h>
-#endif  // !_WIN32_WCE
-#include <stddef.h>
-#include <stdlib.h>  // For strtoll/_strtoul64/malloc/free.
-#include <string.h>  // For memmove.
-
-#include <algorithm>
-#include <string>
-#include <vector>
-
-
-#if GTEST_CAN_STREAM_RESULTS_
-# include <arpa/inet.h>  // NOLINT
-# include <netdb.h>  // NOLINT
-#endif
-
-#if GTEST_OS_WINDOWS
-# include <windows.h>  // NOLINT
-#endif  // GTEST_OS_WINDOWS
-
-
-namespace testing {
-
-// Declares the flags.
-//
-// We don't want the users to modify this flag in the code, but want
-// Google Test's own unit tests to be able to access it. Therefore we
-// declare it here as opposed to in gtest.h.
-GTEST_DECLARE_bool_(death_test_use_fork);
-
-namespace internal {
-
-// The value of GetTestTypeId() as seen from within the Google Test
-// library.  This is solely for testing GetTestTypeId().
-GTEST_API_ extern const TypeId kTestTypeIdInGoogleTest;
-
-// Names of the flags (needed for parsing Google Test flags).
-const char kAlsoRunDisabledTestsFlag[] = "also_run_disabled_tests";
-const char kBreakOnFailureFlag[] = "break_on_failure";
-const char kCatchExceptionsFlag[] = "catch_exceptions";
-const char kColorFlag[] = "color";
-const char kFilterFlag[] = "filter";
-const char kListTestsFlag[] = "list_tests";
-const char kOutputFlag[] = "output";
-const char kPrintTimeFlag[] = "print_time";
-const char kRandomSeedFlag[] = "random_seed";
-const char kRepeatFlag[] = "repeat";
-const char kShuffleFlag[] = "shuffle";
-const char kStackTraceDepthFlag[] = "stack_trace_depth";
-const char kStreamResultToFlag[] = "stream_result_to";
-const char kThrowOnFailureFlag[] = "throw_on_failure";
-const char kFlagfileFlag[] = "flagfile";
-
-// A valid random seed must be in [1, kMaxRandomSeed].
-const int kMaxRandomSeed = 99999;
-
-// g_help_flag is true iff the --help flag or an equivalent form is
-// specified on the command line.
-GTEST_API_ extern bool g_help_flag;
-
-// Returns the current time in milliseconds.
-GTEST_API_ TimeInMillis GetTimeInMillis();
-
-// Returns true iff Google Test should use colors in the output.
-GTEST_API_ bool ShouldUseColor(bool stdout_is_tty);
-
-// Formats the given time in milliseconds as seconds.
-GTEST_API_ std::string FormatTimeInMillisAsSeconds(TimeInMillis ms);
-
-// Converts the given time in milliseconds to a date string in the ISO 8601
-// format, without the timezone information.  N.B.: due to the use the
-// non-reentrant localtime() function, this function is not thread safe.  Do
-// not use it in any code that can be called from multiple threads.
-GTEST_API_ std::string FormatEpochTimeInMillisAsIso8601(TimeInMillis ms);
-
-// Parses a string for an Int32 flag, in the form of "--flag=value".
-//
-// On success, stores the value of the flag in *value, and returns
-// true.  On failure, returns false without changing *value.
-GTEST_API_ bool ParseInt32Flag(
-    const char* str, const char* flag, Int32* value);
-
-// Returns a random seed in range [1, kMaxRandomSeed] based on the
-// given --gtest_random_seed flag value.
-inline int GetRandomSeedFromFlag(Int32 random_seed_flag) {
-  const unsigned int raw_seed = (random_seed_flag == 0) ?
-      static_cast<unsigned int>(GetTimeInMillis()) :
-      static_cast<unsigned int>(random_seed_flag);
-
-  // Normalizes the actual seed to range [1, kMaxRandomSeed] such that
-  // it's easy to type.
-  const int normalized_seed =
-      static_cast<int>((raw_seed - 1U) %
-                       static_cast<unsigned int>(kMaxRandomSeed)) + 1;
-  return normalized_seed;
-}
-
-// Returns the first valid random seed after 'seed'.  The behavior is
-// undefined if 'seed' is invalid.  The seed after kMaxRandomSeed is
-// considered to be 1.
-inline int GetNextRandomSeed(int seed) {
-  GTEST_CHECK_(1 <= seed && seed <= kMaxRandomSeed)
-      << "Invalid random seed " << seed << " - must be in [1, "
-      << kMaxRandomSeed << "].";
-  const int next_seed = seed + 1;
-  return (next_seed > kMaxRandomSeed) ? 1 : next_seed;
-}
-
-// This class saves the values of all Google Test flags in its c'tor, and
-// restores them in its d'tor.
-class GTestFlagSaver {
- public:
-  // The c'tor.
-  GTestFlagSaver() {
-    also_run_disabled_tests_ = GTEST_FLAG(also_run_disabled_tests);
-    break_on_failure_ = GTEST_FLAG(break_on_failure);
-    catch_exceptions_ = GTEST_FLAG(catch_exceptions);
-    color_ = GTEST_FLAG(color);
-    death_test_style_ = GTEST_FLAG(death_test_style);
-    death_test_use_fork_ = GTEST_FLAG(death_test_use_fork);
-    filter_ = GTEST_FLAG(filter);
-    internal_run_death_test_ = GTEST_FLAG(internal_run_death_test);
-    list_tests_ = GTEST_FLAG(list_tests);
-    output_ = GTEST_FLAG(output);
-    print_time_ = GTEST_FLAG(print_time);
-    random_seed_ = GTEST_FLAG(random_seed);
-    repeat_ = GTEST_FLAG(repeat);
-    shuffle_ = GTEST_FLAG(shuffle);
-    stack_trace_depth_ = GTEST_FLAG(stack_trace_depth);
-    stream_result_to_ = GTEST_FLAG(stream_result_to);
-    throw_on_failure_ = GTEST_FLAG(throw_on_failure);
-  }
-
-  // The d'tor is not virtual.  DO NOT INHERIT FROM THIS CLASS.
-  ~GTestFlagSaver() {
-    GTEST_FLAG(also_run_disabled_tests) = also_run_disabled_tests_;
-    GTEST_FLAG(break_on_failure) = break_on_failure_;
-    GTEST_FLAG(catch_exceptions) = catch_exceptions_;
-    GTEST_FLAG(color) = color_;
-    GTEST_FLAG(death_test_style) = death_test_style_;
-    GTEST_FLAG(death_test_use_fork) = death_test_use_fork_;
-    GTEST_FLAG(filter) = filter_;
-    GTEST_FLAG(internal_run_death_test) = internal_run_death_test_;
-    GTEST_FLAG(list_tests) = list_tests_;
-    GTEST_FLAG(output) = output_;
-    GTEST_FLAG(print_time) = print_time_;
-    GTEST_FLAG(random_seed) = random_seed_;
-    GTEST_FLAG(repeat) = repeat_;
-    GTEST_FLAG(shuffle) = shuffle_;
-    GTEST_FLAG(stack_trace_depth) = stack_trace_depth_;
-    GTEST_FLAG(stream_result_to) = stream_result_to_;
-    GTEST_FLAG(throw_on_failure) = throw_on_failure_;
-  }
-
- private:
-  // Fields for saving the original values of flags.
-  bool also_run_disabled_tests_;
-  bool break_on_failure_;
-  bool catch_exceptions_;
-  std::string color_;
-  std::string death_test_style_;
-  bool death_test_use_fork_;
-  std::string filter_;
-  std::string internal_run_death_test_;
-  bool list_tests_;
-  std::string output_;
-  bool print_time_;
-  internal::Int32 random_seed_;
-  internal::Int32 repeat_;
-  bool shuffle_;
-  internal::Int32 stack_trace_depth_;
-  std::string stream_result_to_;
-  bool throw_on_failure_;
-} GTEST_ATTRIBUTE_UNUSED_;
-
-// Converts a Unicode code point to a narrow string in UTF-8 encoding.
-// code_point parameter is of type UInt32 because wchar_t may not be
-// wide enough to contain a code point.
-// If the code_point is not a valid Unicode code point
-// (i.e. outside of Unicode range U+0 to U+10FFFF) it will be converted
-// to "(Invalid Unicode 0xXXXXXXXX)".
-GTEST_API_ std::string CodePointToUtf8(UInt32 code_point);
-
-// Converts a wide string to a narrow string in UTF-8 encoding.
-// The wide string is assumed to have the following encoding:
-//   UTF-16 if sizeof(wchar_t) == 2 (on Windows, Cygwin, Symbian OS)
-//   UTF-32 if sizeof(wchar_t) == 4 (on Linux)
-// Parameter str points to a null-terminated wide string.
-// Parameter num_chars may additionally limit the number
-// of wchar_t characters processed. -1 is used when the entire string
-// should be processed.
-// If the string contains code points that are not valid Unicode code points
-// (i.e. outside of Unicode range U+0 to U+10FFFF) they will be output
-// as '(Invalid Unicode 0xXXXXXXXX)'. If the string is in UTF16 encoding
-// and contains invalid UTF-16 surrogate pairs, values in those pairs
-// will be encoded as individual Unicode characters from Basic Normal Plane.
-GTEST_API_ std::string WideStringToUtf8(const wchar_t* str, int num_chars);
-
-// Reads the GTEST_SHARD_STATUS_FILE environment variable, and creates the file
-// if the variable is present. If a file already exists at this location, this
-// function will write over it. If the variable is present, but the file cannot
-// be created, prints an error and exits.
-void WriteToShardStatusFileIfNeeded();
-
-// Checks whether sharding is enabled by examining the relevant
-// environment variable values. If the variables are present,
-// but inconsistent (e.g., shard_index >= total_shards), prints
-// an error and exits. If in_subprocess_for_death_test, sharding is
-// disabled because it must only be applied to the original test
-// process. Otherwise, we could filter out death tests we intended to execute.
-GTEST_API_ bool ShouldShard(const char* total_shards_str,
-                            const char* shard_index_str,
-                            bool in_subprocess_for_death_test);
-
-// Parses the environment variable var as an Int32. If it is unset,
-// returns default_val. If it is not an Int32, prints an error and
-// and aborts.
-GTEST_API_ Int32 Int32FromEnvOrDie(const char* env_var, Int32 default_val);
-
-// Given the total number of shards, the shard index, and the test id,
-// returns true iff the test should be run on this shard. The test id is
-// some arbitrary but unique non-negative integer assigned to each test
-// method. Assumes that 0 <= shard_index < total_shards.
-GTEST_API_ bool ShouldRunTestOnShard(
-    int total_shards, int shard_index, int test_id);
-
-// STL container utilities.
-
-// Returns the number of elements in the given container that satisfy
-// the given predicate.
-template <class Container, typename Predicate>
-inline int CountIf(const Container& c, Predicate predicate) {
-  // Implemented as an explicit loop since std::count_if() in libCstd on
-  // Solaris has a non-standard signature.
-  int count = 0;
-  for (typename Container::const_iterator it = c.begin(); it != c.end(); ++it) {
-    if (predicate(*it))
-      ++count;
-  }
-  return count;
-}
-
-// Applies a function/functor to each element in the container.
-template <class Container, typename Functor>
-void ForEach(const Container& c, Functor functor) {
-  std::for_each(c.begin(), c.end(), functor);
-}
-
-// Returns the i-th element of the vector, or default_value if i is not
-// in range [0, v.size()).
-template <typename E>
-inline E GetElementOr(const std::vector<E>& v, int i, E default_value) {
-  return (i < 0 || i >= static_cast<int>(v.size())) ? default_value : v[i];
-}
-
-// Performs an in-place shuffle of a range of the vector's elements.
-// 'begin' and 'end' are element indices as an STL-style range;
-// i.e. [begin, end) are shuffled, where 'end' == size() means to
-// shuffle to the end of the vector.
-template <typename E>
-void ShuffleRange(internal::Random* random, int begin, int end,
-                  std::vector<E>* v) {
-  const int size = static_cast<int>(v->size());
-  GTEST_CHECK_(0 <= begin && begin <= size)
-      << "Invalid shuffle range start " << begin << ": must be in range [0, "
-      << size << "].";
-  GTEST_CHECK_(begin <= end && end <= size)
-      << "Invalid shuffle range finish " << end << ": must be in range ["
-      << begin << ", " << size << "].";
-
-  // Fisher-Yates shuffle, from
-  // http://en.wikipedia.org/wiki/Fisher-Yates_shuffle
-  for (int range_width = end - begin; range_width >= 2; range_width--) {
-    const int last_in_range = begin + range_width - 1;
-    const int selected = begin + random->Generate(range_width);
-    std::swap((*v)[selected], (*v)[last_in_range]);
-  }
-}
-
-// Performs an in-place shuffle of the vector's elements.
-template <typename E>
-inline void Shuffle(internal::Random* random, std::vector<E>* v) {
-  ShuffleRange(random, 0, static_cast<int>(v->size()), v);
-}
-
-// A function for deleting an object.  Handy for being used as a
-// functor.
-template <typename T>
-static void Delete(T* x) {
-  delete x;
-}
-
-// A predicate that checks the key of a TestProperty against a known key.
-//
-// TestPropertyKeyIs is copyable.
-class TestPropertyKeyIs {
- public:
-  // Constructor.
-  //
-  // TestPropertyKeyIs has NO default constructor.
-  explicit TestPropertyKeyIs(const std::string& key) : key_(key) {}
-
-  // Returns true iff the test name of test property matches on key_.
-  bool operator()(const TestProperty& test_property) const {
-    return test_property.key() == key_;
-  }
-
- private:
-  std::string key_;
-};
-
-// Class UnitTestOptions.
-//
-// This class contains functions for processing options the user
-// specifies when running the tests.  It has only static members.
-//
-// In most cases, the user can specify an option using either an
-// environment variable or a command line flag.  E.g. you can set the
-// test filter using either GTEST_FILTER or --gtest_filter.  If both
-// the variable and the flag are present, the latter overrides the
-// former.
-class GTEST_API_ UnitTestOptions {
- public:
-  // Functions for processing the gtest_output flag.
-
-  // Returns the output format, or "" for normal printed output.
-  static std::string GetOutputFormat();
-
-  // Returns the absolute path of the requested output file, or the
-  // default (test_detail.xml in the original working directory) if
-  // none was explicitly specified.
-  static std::string GetAbsolutePathToOutputFile();
-
-  // Functions for processing the gtest_filter flag.
-
-  // Returns true iff the wildcard pattern matches the string.  The
-  // first ':' or '\0' character in pattern marks the end of it.
-  //
-  // This recursive algorithm isn't very efficient, but is clear and
-  // works well enough for matching test names, which are short.
-  static bool PatternMatchesString(const char *pattern, const char *str);
-
-  // Returns true iff the user-specified filter matches the test case
-  // name and the test name.
-  static bool FilterMatchesTest(const std::string &test_case_name,
-                                const std::string &test_name);
-
-#if GTEST_OS_WINDOWS
-  // Function for supporting the gtest_catch_exception flag.
-
-  // Returns EXCEPTION_EXECUTE_HANDLER if Google Test should handle the
-  // given SEH exception, or EXCEPTION_CONTINUE_SEARCH otherwise.
-  // This function is useful as an __except condition.
-  static int GTestShouldProcessSEH(DWORD exception_code);
-#endif  // GTEST_OS_WINDOWS
-
-  // Returns true if "name" matches the ':' separated list of glob-style
-  // filters in "filter".
-  static bool MatchesFilter(const std::string& name, const char* filter);
-};
-
-// Returns the current application's name, removing directory path if that
-// is present.  Used by UnitTestOptions::GetOutputFile.
-GTEST_API_ FilePath GetCurrentExecutableName();
-
-// The role interface for getting the OS stack trace as a string.
-class OsStackTraceGetterInterface {
- public:
-  OsStackTraceGetterInterface() {}
-  virtual ~OsStackTraceGetterInterface() {}
-
-  // Returns the current OS stack trace as an std::string.  Parameters:
-  //
-  //   max_depth  - the maximum number of stack frames to be included
-  //                in the trace.
-  //   skip_count - the number of top frames to be skipped; doesn't count
-  //                against max_depth.
-  virtual string CurrentStackTrace(int max_depth, int skip_count) = 0;
-
-  // UponLeavingGTest() should be called immediately before Google Test calls
-  // user code. It saves some information about the current stack that
-  // CurrentStackTrace() will use to find and hide Google Test stack frames.
-  virtual void UponLeavingGTest() = 0;
-
-  // This string is inserted in place of stack frames that are part of
-  // Google Test's implementation.
-  static const char* const kElidedFramesMarker;
-
- private:
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(OsStackTraceGetterInterface);
-};
-
-// A working implementation of the OsStackTraceGetterInterface interface.
-class OsStackTraceGetter : public OsStackTraceGetterInterface {
- public:
-  OsStackTraceGetter() {}
-
-  virtual string CurrentStackTrace(int max_depth, int skip_count);
-  virtual void UponLeavingGTest();
-
- private:
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(OsStackTraceGetter);
-};
-
-// Information about a Google Test trace point.
-struct TraceInfo {
-  const char* file;
-  int line;
-  std::string message;
-};
-
-// This is the default global test part result reporter used in UnitTestImpl.
-// This class should only be used by UnitTestImpl.
-class DefaultGlobalTestPartResultReporter
-  : public TestPartResultReporterInterface {
- public:
-  explicit DefaultGlobalTestPartResultReporter(UnitTestImpl* unit_test);
-  // Implements the TestPartResultReporterInterface. Reports the test part
-  // result in the current test.
-  virtual void ReportTestPartResult(const TestPartResult& result);
-
- private:
-  UnitTestImpl* const unit_test_;
-
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(DefaultGlobalTestPartResultReporter);
-};
-
-// This is the default per thread test part result reporter used in
-// UnitTestImpl. This class should only be used by UnitTestImpl.
-class DefaultPerThreadTestPartResultReporter
-    : public TestPartResultReporterInterface {
- public:
-  explicit DefaultPerThreadTestPartResultReporter(UnitTestImpl* unit_test);
-  // Implements the TestPartResultReporterInterface. The implementation just
-  // delegates to the current global test part result reporter of *unit_test_.
-  virtual void ReportTestPartResult(const TestPartResult& result);
-
- private:
-  UnitTestImpl* const unit_test_;
-
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(DefaultPerThreadTestPartResultReporter);
-};
-
-// The private implementation of the UnitTest class.  We don't protect
-// the methods under a mutex, as this class is not accessible by a
-// user and the UnitTest class that delegates work to this class does
-// proper locking.
-class GTEST_API_ UnitTestImpl {
- public:
-  explicit UnitTestImpl(UnitTest* parent);
-  virtual ~UnitTestImpl();
-
-  // There are two different ways to register your own TestPartResultReporter.
-  // You can register your own repoter to listen either only for test results
-  // from the current thread or for results from all threads.
-  // By default, each per-thread test result repoter just passes a new
-  // TestPartResult to the global test result reporter, which registers the
-  // test part result for the currently running test.
-
-  // Returns the global test part result reporter.
-  TestPartResultReporterInterface* GetGlobalTestPartResultReporter();
-
-  // Sets the global test part result reporter.
-  void SetGlobalTestPartResultReporter(
-      TestPartResultReporterInterface* reporter);
-
-  // Returns the test part result reporter for the current thread.
-  TestPartResultReporterInterface* GetTestPartResultReporterForCurrentThread();
-
-  // Sets the test part result reporter for the current thread.
-  void SetTestPartResultReporterForCurrentThread(
-      TestPartResultReporterInterface* reporter);
-
-  // Gets the number of successful test cases.
-  int successful_test_case_count() const;
-
-  // Gets the number of failed test cases.
-  int failed_test_case_count() const;
-
-  // Gets the number of all test cases.
-  int total_test_case_count() const;
-
-  // Gets the number of all test cases that contain at least one test
-  // that should run.
-  int test_case_to_run_count() const;
-
-  // Gets the number of successful tests.
-  int successful_test_count() const;
-
-  // Gets the number of failed tests.
-  int failed_test_count() const;
-
-  // Gets the number of disabled tests that will be reported in the XML report.
-  int reportable_disabled_test_count() const;
-
-  // Gets the number of disabled tests.
-  int disabled_test_count() const;
-
-  // Gets the number of tests to be printed in the XML report.
-  int reportable_test_count() const;
-
-  // Gets the number of all tests.
-  int total_test_count() const;
-
-  // Gets the number of tests that should run.
-  int test_to_run_count() const;
-
-  // Gets the time of the test program start, in ms from the start of the
-  // UNIX epoch.
-  TimeInMillis start_timestamp() const { return start_timestamp_; }
-
-  // Gets the elapsed time, in milliseconds.
-  TimeInMillis elapsed_time() const { return elapsed_time_; }
-
-  // Returns true iff the unit test passed (i.e. all test cases passed).
-  bool Passed() const { return !Failed(); }
-
-  // Returns true iff the unit test failed (i.e. some test case failed
-  // or something outside of all tests failed).
-  bool Failed() const {
-    return failed_test_case_count() > 0 || ad_hoc_test_result()->Failed();
-  }
-
-  // Gets the i-th test case among all the test cases. i can range from 0 to
-  // total_test_case_count() - 1. If i is not in that range, returns NULL.
-  const TestCase* GetTestCase(int i) const {
-    const int index = GetElementOr(test_case_indices_, i, -1);
-    return index < 0 ? NULL : test_cases_[i];
-  }
-
-  // Gets the i-th test case among all the test cases. i can range from 0 to
-  // total_test_case_count() - 1. If i is not in that range, returns NULL.
-  TestCase* GetMutableTestCase(int i) {
-    const int index = GetElementOr(test_case_indices_, i, -1);
-    return index < 0 ? NULL : test_cases_[index];
-  }
-
-  // Provides access to the event listener list.
-  TestEventListeners* listeners() { return &listeners_; }
-
-  // Returns the TestResult for the test that's currently running, or
-  // the TestResult for the ad hoc test if no test is running.
-  TestResult* current_test_result();
-
-  // Returns the TestResult for the ad hoc test.
-  const TestResult* ad_hoc_test_result() const { return &ad_hoc_test_result_; }
-
-  // Sets the OS stack trace getter.
-  //
-  // Does nothing if the input and the current OS stack trace getter
-  // are the same; otherwise, deletes the old getter and makes the
-  // input the current getter.
-  void set_os_stack_trace_getter(OsStackTraceGetterInterface* getter);
-
-  // Returns the current OS stack trace getter if it is not NULL;
-  // otherwise, creates an OsStackTraceGetter, makes it the current
-  // getter, and returns it.
-  OsStackTraceGetterInterface* os_stack_trace_getter();
-
-  // Returns the current OS stack trace as an std::string.
-  //
-  // The maximum number of stack frames to be included is specified by
-  // the gtest_stack_trace_depth flag.  The skip_count parameter
-  // specifies the number of top frames to be skipped, which doesn't
-  // count against the number of frames to be included.
-  //
-  // For example, if Foo() calls Bar(), which in turn calls
-  // CurrentOsStackTraceExceptTop(1), Foo() will be included in the
-  // trace but Bar() and CurrentOsStackTraceExceptTop() won't.
-  std::string CurrentOsStackTraceExceptTop(int skip_count) GTEST_NO_INLINE_;
-
-  // Finds and returns a TestCase with the given name.  If one doesn't
-  // exist, creates one and returns it.
-  //
-  // Arguments:
-  //
-  //   test_case_name: name of the test case
-  //   type_param:     the name of the test's type parameter, or NULL if
-  //                   this is not a typed or a type-parameterized test.
-  //   set_up_tc:      pointer to the function that sets up the test case
-  //   tear_down_tc:   pointer to the function that tears down the test case
-  TestCase* GetTestCase(const char* test_case_name,
-                        const char* type_param,
-                        Test::SetUpTestCaseFunc set_up_tc,
-                        Test::TearDownTestCaseFunc tear_down_tc);
-
-  // Adds a TestInfo to the unit test.
-  //
-  // Arguments:
-  //
-  //   set_up_tc:    pointer to the function that sets up the test case
-  //   tear_down_tc: pointer to the function that tears down the test case
-  //   test_info:    the TestInfo object
-  void AddTestInfo(Test::SetUpTestCaseFunc set_up_tc,
-                   Test::TearDownTestCaseFunc tear_down_tc,
-                   TestInfo* test_info) {
-    // In order to support thread-safe death tests, we need to
-    // remember the original working directory when the test program
-    // was first invoked.  We cannot do this in RUN_ALL_TESTS(), as
-    // the user may have changed the current directory before calling
-    // RUN_ALL_TESTS().  Therefore we capture the current directory in
-    // AddTestInfo(), which is called to register a TEST or TEST_F
-    // before main() is reached.
-    if (original_working_dir_.IsEmpty()) {
-      original_working_dir_.Set(FilePath::GetCurrentDir());
-      GTEST_CHECK_(!original_working_dir_.IsEmpty())
-          << "Failed to get the current working directory.";
-    }
-
-    GetTestCase(test_info->test_case_name(),
-                test_info->type_param(),
-                set_up_tc,
-                tear_down_tc)->AddTestInfo(test_info);
-  }
-
-#if GTEST_HAS_PARAM_TEST
-  // Returns ParameterizedTestCaseRegistry object used to keep track of
-  // value-parameterized tests and instantiate and register them.
-  internal::ParameterizedTestCaseRegistry& parameterized_test_registry() {
-    return parameterized_test_registry_;
-  }
-#endif  // GTEST_HAS_PARAM_TEST
-
-  // Sets the TestCase object for the test that's currently running.
-  void set_current_test_case(TestCase* a_current_test_case) {
-    current_test_case_ = a_current_test_case;
-  }
-
-  // Sets the TestInfo object for the test that's currently running.  If
-  // current_test_info is NULL, the assertion results will be stored in
-  // ad_hoc_test_result_.
-  void set_current_test_info(TestInfo* a_current_test_info) {
-    current_test_info_ = a_current_test_info;
-  }
-
-  // Registers all parameterized tests defined using TEST_P and
-  // INSTANTIATE_TEST_CASE_P, creating regular tests for each test/parameter
-  // combination. This method can be called more then once; it has guards
-  // protecting from registering the tests more then once.  If
-  // value-parameterized tests are disabled, RegisterParameterizedTests is
-  // present but does nothing.
-  void RegisterParameterizedTests();
-
-  // Runs all tests in this UnitTest object, prints the result, and
-  // returns true if all tests are successful.  If any exception is
-  // thrown during a test, this test is considered to be failed, but
-  // the rest of the tests will still be run.
-  bool RunAllTests();
-
-  // Clears the results of all tests, except the ad hoc tests.
-  void ClearNonAdHocTestResult() {
-    ForEach(test_cases_, TestCase::ClearTestCaseResult);
-  }
-
-  // Clears the results of ad-hoc test assertions.
-  void ClearAdHocTestResult() {
-    ad_hoc_test_result_.Clear();
-  }
-
-  // Adds a TestProperty to the current TestResult object when invoked in a
-  // context of a test or a test case, or to the global property set. If the
-  // result already contains a property with the same key, the value will be
-  // updated.
-  void RecordProperty(const TestProperty& test_property);
-
-  enum ReactionToSharding {
-    HONOR_SHARDING_PROTOCOL,
-    IGNORE_SHARDING_PROTOCOL
-  };
-
-  // Matches the full name of each test against the user-specified
-  // filter to decide whether the test should run, then records the
-  // result in each TestCase and TestInfo object.
-  // If shard_tests == HONOR_SHARDING_PROTOCOL, further filters tests
-  // based on sharding variables in the environment.
-  // Returns the number of tests that should run.
-  int FilterTests(ReactionToSharding shard_tests);
-
-  // Prints the names of the tests matching the user-specified filter flag.
-  void ListTestsMatchingFilter();
-
-  const TestCase* current_test_case() const { return current_test_case_; }
-  TestInfo* current_test_info() { return current_test_info_; }
-  const TestInfo* current_test_info() const { return current_test_info_; }
-
-  // Returns the vector of environments that need to be set-up/torn-down
-  // before/after the tests are run.
-  std::vector<Environment*>& environments() { return environments_; }
-
-  // Getters for the per-thread Google Test trace stack.
-  std::vector<TraceInfo>& gtest_trace_stack() {
-    return *(gtest_trace_stack_.pointer());
-  }
-  const std::vector<TraceInfo>& gtest_trace_stack() const {
-    return gtest_trace_stack_.get();
-  }
-
-#if GTEST_HAS_DEATH_TEST
-  void InitDeathTestSubprocessControlInfo() {
-    internal_run_death_test_flag_.reset(ParseInternalRunDeathTestFlag());
-  }
-  // Returns a pointer to the parsed --gtest_internal_run_death_test
-  // flag, or NULL if that flag was not specified.
-  // This information is useful only in a death test child process.
-  // Must not be called before a call to InitGoogleTest.
-  const InternalRunDeathTestFlag* internal_run_death_test_flag() const {
-    return internal_run_death_test_flag_.get();
-  }
-
-  // Returns a pointer to the current death test factory.
-  internal::DeathTestFactory* death_test_factory() {
-    return death_test_factory_.get();
-  }
-
-  void SuppressTestEventsIfInSubprocess();
-
-  friend class ReplaceDeathTestFactory;
-#endif  // GTEST_HAS_DEATH_TEST
-
-  // Initializes the event listener performing XML output as specified by
-  // UnitTestOptions. Must not be called before InitGoogleTest.
-  void ConfigureXmlOutput();
-
-#if GTEST_CAN_STREAM_RESULTS_
-  // Initializes the event listener for streaming test results to a socket.
-  // Must not be called before InitGoogleTest.
-  void ConfigureStreamingOutput();
-#endif
-
-  // Performs initialization dependent upon flag values obtained in
-  // ParseGoogleTestFlagsOnly.  Is called from InitGoogleTest after the call to
-  // ParseGoogleTestFlagsOnly.  In case a user neglects to call InitGoogleTest
-  // this function is also called from RunAllTests.  Since this function can be
-  // called more than once, it has to be idempotent.
-  void PostFlagParsingInit();
-
-  // Gets the random seed used at the start of the current test iteration.
-  int random_seed() const { return random_seed_; }
-
-  // Gets the random number generator.
-  internal::Random* random() { return &random_; }
-
-  // Shuffles all test cases, and the tests within each test case,
-  // making sure that death tests are still run first.
-  void ShuffleTests();
-
-  // Restores the test cases and tests to their order before the first shuffle.
-  void UnshuffleTests();
-
-  // Returns the value of GTEST_FLAG(catch_exceptions) at the moment
-  // UnitTest::Run() starts.
-  bool catch_exceptions() const { return catch_exceptions_; }
-
- private:
-  friend class ::testing::UnitTest;
-
-  // Used by UnitTest::Run() to capture the state of
-  // GTEST_FLAG(catch_exceptions) at the moment it starts.
-  void set_catch_exceptions(bool value) { catch_exceptions_ = value; }
-
-  // The UnitTest object that owns this implementation object.
-  UnitTest* const parent_;
-
-  // The working directory when the first TEST() or TEST_F() was
-  // executed.
-  internal::FilePath original_working_dir_;
-
-  // The default test part result reporters.
-  DefaultGlobalTestPartResultReporter default_global_test_part_result_reporter_;
-  DefaultPerThreadTestPartResultReporter
-      default_per_thread_test_part_result_reporter_;
-
-  // Points to (but doesn't own) the global test part result reporter.
-  TestPartResultReporterInterface* global_test_part_result_repoter_;
-
-  // Protects read and write access to global_test_part_result_reporter_.
-  internal::Mutex global_test_part_result_reporter_mutex_;
-
-  // Points to (but doesn't own) the per-thread test part result reporter.
-  internal::ThreadLocal<TestPartResultReporterInterface*>
-      per_thread_test_part_result_reporter_;
-
-  // The vector of environments that need to be set-up/torn-down
-  // before/after the tests are run.
-  std::vector<Environment*> environments_;
-
-  // The vector of TestCases in their original order.  It owns the
-  // elements in the vector.
-  std::vector<TestCase*> test_cases_;
-
-  // Provides a level of indirection for the test case list to allow
-  // easy shuffling and restoring the test case order.  The i-th
-  // element of this vector is the index of the i-th test case in the
-  // shuffled order.
-  std::vector<int> test_case_indices_;
-
-#if GTEST_HAS_PARAM_TEST
-  // ParameterizedTestRegistry object used to register value-parameterized
-  // tests.
-  internal::ParameterizedTestCaseRegistry parameterized_test_registry_;
-
-  // Indicates whether RegisterParameterizedTests() has been called already.
-  bool parameterized_tests_registered_;
-#endif  // GTEST_HAS_PARAM_TEST
-
-  // Index of the last death test case registered.  Initially -1.
-  int last_death_test_case_;
-
-  // This points to the TestCase for the currently running test.  It
-  // changes as Google Test goes through one test case after another.
-  // When no test is running, this is set to NULL and Google Test
-  // stores assertion results in ad_hoc_test_result_.  Initially NULL.
-  TestCase* current_test_case_;
-
-  // This points to the TestInfo for the currently running test.  It
-  // changes as Google Test goes through one test after another.  When
-  // no test is running, this is set to NULL and Google Test stores
-  // assertion results in ad_hoc_test_result_.  Initially NULL.
-  TestInfo* current_test_info_;
-
-  // Normally, a user only writes assertions inside a TEST or TEST_F,
-  // or inside a function called by a TEST or TEST_F.  Since Google
-  // Test keeps track of which test is current running, it can
-  // associate such an assertion with the test it belongs to.
-  //
-  // If an assertion is encountered when no TEST or TEST_F is running,
-  // Google Test attributes the assertion result to an imaginary "ad hoc"
-  // test, and records the result in ad_hoc_test_result_.
-  TestResult ad_hoc_test_result_;
-
-  // The list of event listeners that can be used to track events inside
-  // Google Test.
-  TestEventListeners listeners_;
-
-  // The OS stack trace getter.  Will be deleted when the UnitTest
-  // object is destructed.  By default, an OsStackTraceGetter is used,
-  // but the user can set this field to use a custom getter if that is
-  // desired.
-  OsStackTraceGetterInterface* os_stack_trace_getter_;
-
-  // True iff PostFlagParsingInit() has been called.
-  bool post_flag_parse_init_performed_;
-
-  // The random number seed used at the beginning of the test run.
-  int random_seed_;
-
-  // Our random number generator.
-  internal::Random random_;
-
-  // The time of the test program start, in ms from the start of the
-  // UNIX epoch.
-  TimeInMillis start_timestamp_;
-
-  // How long the test took to run, in milliseconds.
-  TimeInMillis elapsed_time_;
-
-#if GTEST_HAS_DEATH_TEST
-  // The decomposed components of the gtest_internal_run_death_test flag,
-  // parsed when RUN_ALL_TESTS is called.
-  internal::scoped_ptr<InternalRunDeathTestFlag> internal_run_death_test_flag_;
-  internal::scoped_ptr<internal::DeathTestFactory> death_test_factory_;
-#endif  // GTEST_HAS_DEATH_TEST
-
-  // A per-thread stack of traces created by the SCOPED_TRACE() macro.
-  internal::ThreadLocal<std::vector<TraceInfo> > gtest_trace_stack_;
-
-  // The value of GTEST_FLAG(catch_exceptions) at the moment RunAllTests()
-  // starts.
-  bool catch_exceptions_;
-
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(UnitTestImpl);
-};  // class UnitTestImpl
-
-// Convenience function for accessing the global UnitTest
-// implementation object.
-inline UnitTestImpl* GetUnitTestImpl() {
-  return UnitTest::GetInstance()->impl();
-}
-
-#if GTEST_USES_SIMPLE_RE
-
-// Internal helper functions for implementing the simple regular
-// expression matcher.
-GTEST_API_ bool IsInSet(char ch, const char* str);
-GTEST_API_ bool IsAsciiDigit(char ch);
-GTEST_API_ bool IsAsciiPunct(char ch);
-GTEST_API_ bool IsRepeat(char ch);
-GTEST_API_ bool IsAsciiWhiteSpace(char ch);
-GTEST_API_ bool IsAsciiWordChar(char ch);
-GTEST_API_ bool IsValidEscape(char ch);
-GTEST_API_ bool AtomMatchesChar(bool escaped, char pattern, char ch);
-GTEST_API_ bool ValidateRegex(const char* regex);
-GTEST_API_ bool MatchRegexAtHead(const char* regex, const char* str);
-GTEST_API_ bool MatchRepetitionAndRegexAtHead(
-    bool escaped, char ch, char repeat, const char* regex, const char* str);
-GTEST_API_ bool MatchRegexAnywhere(const char* regex, const char* str);
-
-#endif  // GTEST_USES_SIMPLE_RE
-
-// Parses the command line for Google Test flags, without initializing
-// other parts of Google Test.
-GTEST_API_ void ParseGoogleTestFlagsOnly(int* argc, char** argv);
-GTEST_API_ void ParseGoogleTestFlagsOnly(int* argc, wchar_t** argv);
-
-#if GTEST_HAS_DEATH_TEST
-
-// Returns the message describing the last system error, regardless of the
-// platform.
-GTEST_API_ std::string GetLastErrnoDescription();
-
-// Attempts to parse a string into a positive integer pointed to by the
-// number parameter.  Returns true if that is possible.
-// GTEST_HAS_DEATH_TEST implies that we have ::std::string, so we can use
-// it here.
-template <typename Integer>
-bool ParseNaturalNumber(const ::std::string& str, Integer* number) {
-  // Fail fast if the given string does not begin with a digit;
-  // this bypasses strtoXXX's "optional leading whitespace and plus
-  // or minus sign" semantics, which are undesirable here.
-  if (str.empty() || !IsDigit(str[0])) {
-    return false;
-  }
-  errno = 0;
-
-  char* end;
-  // BiggestConvertible is the largest integer type that system-provided
-  // string-to-number conversion routines can return.
-
-# if GTEST_OS_WINDOWS && !defined(__GNUC__)
-
-  // MSVC and C++ Builder define __int64 instead of the standard long long.
-  typedef unsigned __int64 BiggestConvertible;
-  const BiggestConvertible parsed = _strtoui64(str.c_str(), &end, 10);
-
-# else
-
-  typedef unsigned long long BiggestConvertible;  // NOLINT
-  const BiggestConvertible parsed = strtoull(str.c_str(), &end, 10);
-
-# endif  // GTEST_OS_WINDOWS && !defined(__GNUC__)
-
-  const bool parse_success = *end == '\0' && errno == 0;
-
-  // TODO(vladl@google.com): Convert this to compile time assertion when it is
-  // available.
-  GTEST_CHECK_(sizeof(Integer) <= sizeof(parsed));
-
-  const Integer result = static_cast<Integer>(parsed);
-  if (parse_success && static_cast<BiggestConvertible>(result) == parsed) {
-    *number = result;
-    return true;
-  }
-  return false;
-}
-#endif  // GTEST_HAS_DEATH_TEST
-
-// TestResult contains some private methods that should be hidden from
-// Google Test user but are required for testing. This class allow our tests
-// to access them.
-//
-// This class is supplied only for the purpose of testing Google Test's own
-// constructs. Do not use it in user tests, either directly or indirectly.
-class TestResultAccessor {
- public:
-  static void RecordProperty(TestResult* test_result,
-                             const std::string& xml_element,
-                             const TestProperty& property) {
-    test_result->RecordProperty(xml_element, property);
-  }
-
-  static void ClearTestPartResults(TestResult* test_result) {
-    test_result->ClearTestPartResults();
-  }
-
-  static const std::vector<testing::TestPartResult>& test_part_results(
-      const TestResult& test_result) {
-    return test_result.test_part_results();
-  }
-};
-
-#if GTEST_CAN_STREAM_RESULTS_
-
-// Streams test results to the given port on the given host machine.
-class GTEST_API_ StreamingListener : public EmptyTestEventListener {
- public:
-  // Abstract base class for writing strings to a socket.
-  class AbstractSocketWriter {
-   public:
-    virtual ~AbstractSocketWriter() {}
-
-    // Sends a string to the socket.
-    virtual void Send(const string& message) = 0;
-
-    // Closes the socket.
-    virtual void CloseConnection() {}
-
-    // Sends a string and a newline to the socket.
-    void SendLn(const string& message) {
-      Send(message + "\n");
-    }
-  };
-
-  // Concrete class for actually writing strings to a socket.
-  class SocketWriter : public AbstractSocketWriter {
-   public:
-    SocketWriter(const string& host, const string& port)
-        : sockfd_(-1), host_name_(host), port_num_(port) {
-      MakeConnection();
-    }
-
-    virtual ~SocketWriter() {
-      if (sockfd_ != -1)
-        CloseConnection();
-    }
-
-    // Sends a string to the socket.
-    virtual void Send(const string& message) {
-      GTEST_CHECK_(sockfd_ != -1)
-          << "Send() can be called only when there is a connection.";
-
-      const int len = static_cast<int>(message.length());
-      if (write(sockfd_, message.c_str(), len) != len) {
-        GTEST_LOG_(WARNING)
-            << "stream_result_to: failed to stream to "
-            << host_name_ << ":" << port_num_;
-      }
-    }
-
-   private:
-    // Creates a client socket and connects to the server.
-    void MakeConnection();
-
-    // Closes the socket.
-    void CloseConnection() {
-      GTEST_CHECK_(sockfd_ != -1)
-          << "CloseConnection() can be called only when there is a connection.";
-
-      close(sockfd_);
-      sockfd_ = -1;
-    }
-
-    int sockfd_;  // socket file descriptor
-    const string host_name_;
-    const string port_num_;
-
-    GTEST_DISALLOW_COPY_AND_ASSIGN_(SocketWriter);
-  };  // class SocketWriter
-
-  // Escapes '=', '&', '%', and '\n' characters in str as "%xx".
-  static string UrlEncode(const char* str);
-
-  StreamingListener(const string& host, const string& port)
-      : socket_writer_(new SocketWriter(host, port)) { Start(); }
-
-  explicit StreamingListener(AbstractSocketWriter* socket_writer)
-      : socket_writer_(socket_writer) { Start(); }
-
-  void OnTestProgramStart(const UnitTest& /* unit_test */) {
-    SendLn("event=TestProgramStart");
-  }
-
-  void OnTestProgramEnd(const UnitTest& unit_test) {
-    // Note that Google Test current only report elapsed time for each
-    // test iteration, not for the entire test program.
-    SendLn("event=TestProgramEnd&passed=" + FormatBool(unit_test.Passed()));
-
-    // Notify the streaming server to stop.
-    socket_writer_->CloseConnection();
-  }
-
-  void OnTestIterationStart(const UnitTest& /* unit_test */, int iteration) {
-    SendLn("event=TestIterationStart&iteration=" +
-           StreamableToString(iteration));
-  }
-
-  void OnTestIterationEnd(const UnitTest& unit_test, int /* iteration */) {
-    SendLn("event=TestIterationEnd&passed=" +
-           FormatBool(unit_test.Passed()) + "&elapsed_time=" +
-           StreamableToString(unit_test.elapsed_time()) + "ms");
-  }
-
-  void OnTestCaseStart(const TestCase& test_case) {
-    SendLn(std::string("event=TestCaseStart&name=") + test_case.name());
-  }
-
-  void OnTestCaseEnd(const TestCase& test_case) {
-    SendLn("event=TestCaseEnd&passed=" + FormatBool(test_case.Passed())
-           + "&elapsed_time=" + StreamableToString(test_case.elapsed_time())
-           + "ms");
-  }
-
-  void OnTestStart(const TestInfo& test_info) {
-    SendLn(std::string("event=TestStart&name=") + test_info.name());
-  }
-
-  void OnTestEnd(const TestInfo& test_info) {
-    SendLn("event=TestEnd&passed=" +
-           FormatBool((test_info.result())->Passed()) +
-           "&elapsed_time=" +
-           StreamableToString((test_info.result())->elapsed_time()) + "ms");
-  }
-
-  void OnTestPartResult(const TestPartResult& test_part_result) {
-    const char* file_name = test_part_result.file_name();
-    if (file_name == NULL)
-      file_name = "";
-    SendLn("event=TestPartResult&file=" + UrlEncode(file_name) +
-           "&line=" + StreamableToString(test_part_result.line_number()) +
-           "&message=" + UrlEncode(test_part_result.message()));
-  }
-
- private:
-  // Sends the given message and a newline to the socket.
-  void SendLn(const string& message) { socket_writer_->SendLn(message); }
-
-  // Called at the start of streaming to notify the receiver what
-  // protocol we are using.
-  void Start() { SendLn("gtest_streaming_protocol_version=1.0"); }
-
-  string FormatBool(bool value) { return value ? "1" : "0"; }
-
-  const scoped_ptr<AbstractSocketWriter> socket_writer_;
-
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(StreamingListener);
-};  // class StreamingListener
-
-#endif  // GTEST_CAN_STREAM_RESULTS_
-
-}  // namespace internal
-}  // namespace testing
-
-#endif  // GTEST_SRC_GTEST_INTERNAL_INL_H_
-#undef GTEST_IMPLEMENTATION_
-
-#if GTEST_OS_WINDOWS
-# define vsnprintf _vsnprintf
-#endif  // GTEST_OS_WINDOWS
-
-namespace testing {
-
-using internal::CountIf;
-using internal::ForEach;
-using internal::GetElementOr;
-using internal::Shuffle;
-
-// Constants.
-
-// A test whose test case name or test name matches this filter is
-// disabled and not run.
-static const char kDisableTestFilter[] = "DISABLED_*:*/DISABLED_*";
-
-// A test case whose name matches this filter is considered a death
-// test case and will be run before test cases whose name doesn't
-// match this filter.
-static const char kDeathTestCaseFilter[] = "*DeathTest:*DeathTest/*";
-
-// A test filter that matches everything.
-static const char kUniversalFilter[] = "*";
-
-// The default output file for XML output.
-static const char kDefaultOutputFile[] = "test_detail.xml";
-
-// The environment variable name for the test shard index.
-static const char kTestShardIndex[] = "GTEST_SHARD_INDEX";
-// The environment variable name for the total number of test shards.
-static const char kTestTotalShards[] = "GTEST_TOTAL_SHARDS";
-// The environment variable name for the test shard status file.
-static const char kTestShardStatusFile[] = "GTEST_SHARD_STATUS_FILE";
-
-namespace internal {
-
-// The text used in failure messages to indicate the start of the
-// stack trace.
-const char kStackTraceMarker[] = "\nStack trace:\n";
-
-// g_help_flag is true iff the --help flag or an equivalent form is
-// specified on the command line.
-bool g_help_flag = false;
-
-}  // namespace internal
-
-static const char* GetDefaultFilter() {
-#ifdef GTEST_TEST_FILTER_ENV_VAR_
-  const char* const testbridge_test_only = getenv(GTEST_TEST_FILTER_ENV_VAR_);
-  if (testbridge_test_only != NULL) {
-    return testbridge_test_only;
-  }
-#endif  // GTEST_TEST_FILTER_ENV_VAR_
-  return kUniversalFilter;
-}
-
-GTEST_DEFINE_bool_(
-    also_run_disabled_tests,
-    internal::BoolFromGTestEnv("also_run_disabled_tests", false),
-    "Run disabled tests too, in addition to the tests normally being run.");
-
-GTEST_DEFINE_bool_(
-    break_on_failure,
-    internal::BoolFromGTestEnv("break_on_failure", false),
-    "True iff a failed assertion should be a debugger break-point.");
-
-GTEST_DEFINE_bool_(
-    catch_exceptions,
-    internal::BoolFromGTestEnv("catch_exceptions", true),
-    "True iff " GTEST_NAME_
-    " should catch exceptions and treat them as test failures.");
-
-GTEST_DEFINE_string_(
-    color,
-    internal::StringFromGTestEnv("color", "auto"),
-    "Whether to use colors in the output.  Valid values: yes, no, "
-    "and auto.  'auto' means to use colors if the output is "
-    "being sent to a terminal and the TERM environment variable "
-    "is set to a terminal type that supports colors.");
-
-GTEST_DEFINE_string_(
-    filter,
-    internal::StringFromGTestEnv("filter", GetDefaultFilter()),
-    "A colon-separated list of glob (not regex) patterns "
-    "for filtering the tests to run, optionally followed by a "
-    "'-' and a : separated list of negative patterns (tests to "
-    "exclude).  A test is run if it matches one of the positive "
-    "patterns and does not match any of the negative patterns.");
-
-GTEST_DEFINE_bool_(list_tests, false,
-                   "List all tests without running them.");
-
-GTEST_DEFINE_string_(
-    output,
-    internal::StringFromGTestEnv("output", ""),
-    "A format (currently must be \"xml\"), optionally followed "
-    "by a colon and an output file name or directory. A directory "
-    "is indicated by a trailing pathname separator. "
-    "Examples: \"xml:filename.xml\", \"xml::directoryname/\". "
-    "If a directory is specified, output files will be created "
-    "within that directory, with file-names based on the test "
-    "executable's name and, if necessary, made unique by adding "
-    "digits.");
-
-GTEST_DEFINE_bool_(
-    print_time,
-    internal::BoolFromGTestEnv("print_time", true),
-    "True iff " GTEST_NAME_
-    " should display elapsed time in text output.");
-
-GTEST_DEFINE_int32_(
-    random_seed,
-    internal::Int32FromGTestEnv("random_seed", 0),
-    "Random number seed to use when shuffling test orders.  Must be in range "
-    "[1, 99999], or 0 to use a seed based on the current time.");
-
-GTEST_DEFINE_int32_(
-    repeat,
-    internal::Int32FromGTestEnv("repeat", 1),
-    "How many times to repeat each test.  Specify a negative number "
-    "for repeating forever.  Useful for shaking out flaky tests.");
-
-GTEST_DEFINE_bool_(
-    show_internal_stack_frames, false,
-    "True iff " GTEST_NAME_ " should include internal stack frames when "
-    "printing test failure stack traces.");
-
-GTEST_DEFINE_bool_(
-    shuffle,
-    internal::BoolFromGTestEnv("shuffle", false),
-    "True iff " GTEST_NAME_
-    " should randomize tests' order on every run.");
-
-GTEST_DEFINE_int32_(
-    stack_trace_depth,
-    internal::Int32FromGTestEnv("stack_trace_depth", kMaxStackTraceDepth),
-    "The maximum number of stack frames to print when an "
-    "assertion fails.  The valid range is 0 through 100, inclusive.");
-
-GTEST_DEFINE_string_(
-    stream_result_to,
-    internal::StringFromGTestEnv("stream_result_to", ""),
-    "This flag specifies the host name and the port number on which to stream "
-    "test results. Example: \"localhost:555\". The flag is effective only on "
-    "Linux.");
-
-GTEST_DEFINE_bool_(
-    throw_on_failure,
-    internal::BoolFromGTestEnv("throw_on_failure", false),
-    "When this flag is specified, a failed assertion will throw an exception "
-    "if exceptions are enabled or exit the program with a non-zero code "
-    "otherwise.");
-
-#if GTEST_USE_OWN_FLAGFILE_FLAG_
-GTEST_DEFINE_string_(
-    flagfile,
-    internal::StringFromGTestEnv("flagfile", ""),
-    "This flag specifies the flagfile to read command-line flags from.");
-#endif  // GTEST_USE_OWN_FLAGFILE_FLAG_
-
-namespace internal {
-
-// Generates a random number from [0, range), using a Linear
-// Congruential Generator (LCG).  Crashes if 'range' is 0 or greater
-// than kMaxRange.
-UInt32 Random::Generate(UInt32 range) {
-  // These constants are the same as are used in glibc's rand(3).
-  state_ = (1103515245U*state_ + 12345U) % kMaxRange;
-
-  GTEST_CHECK_(range > 0)
-      << "Cannot generate a number in the range [0, 0).";
-  GTEST_CHECK_(range <= kMaxRange)
-      << "Generation of a number in [0, " << range << ") was requested, "
-      << "but this can only generate numbers in [0, " << kMaxRange << ").";
-
-  // Converting via modulus introduces a bit of downward bias, but
-  // it's simple, and a linear congruential generator isn't too good
-  // to begin with.
-  return state_ % range;
-}
-
-// GTestIsInitialized() returns true iff the user has initialized
-// Google Test.  Useful for catching the user mistake of not initializing
-// Google Test before calling RUN_ALL_TESTS().
-static bool GTestIsInitialized() { return GetArgvs().size() > 0; }
-
-// Iterates over a vector of TestCases, keeping a running sum of the
-// results of calling a given int-returning method on each.
-// Returns the sum.
-static int SumOverTestCaseList(const std::vector<TestCase*>& case_list,
-                               int (TestCase::*method)() const) {
-  int sum = 0;
-  for (size_t i = 0; i < case_list.size(); i++) {
-    sum += (case_list[i]->*method)();
-  }
-  return sum;
-}
-
-// Returns true iff the test case passed.
-static bool TestCasePassed(const TestCase* test_case) {
-  return test_case->should_run() && test_case->Passed();
-}
-
-// Returns true iff the test case failed.
-static bool TestCaseFailed(const TestCase* test_case) {
-  return test_case->should_run() && test_case->Failed();
-}
-
-// Returns true iff test_case contains at least one test that should
-// run.
-static bool ShouldRunTestCase(const TestCase* test_case) {
-  return test_case->should_run();
-}
-
-// AssertHelper constructor.
-AssertHelper::AssertHelper(TestPartResult::Type type,
-                           const char* file,
-                           int line,
-                           const char* message)
-    : data_(new AssertHelperData(type, file, line, message)) {
-}
-
-AssertHelper::~AssertHelper() {
-  delete data_;
-}
-
-// Message assignment, for assertion streaming support.
-void AssertHelper::operator=(const Message& message) const {
-  UnitTest::GetInstance()->
-    AddTestPartResult(data_->type, data_->file, data_->line,
-                      AppendUserMessage(data_->message, message),
-                      UnitTest::GetInstance()->impl()
-                      ->CurrentOsStackTraceExceptTop(1)
-                      // Skips the stack frame for this function itself.
-                      );  // NOLINT
-}
-
-// Mutex for linked pointers.
-GTEST_API_ GTEST_DEFINE_STATIC_MUTEX_(g_linked_ptr_mutex);
-
-// A copy of all command line arguments.  Set by InitGoogleTest().
-::std::vector<testing::internal::string> g_argvs;
-
-const ::std::vector<testing::internal::string>& GetArgvs() {
-#if defined(GTEST_CUSTOM_GET_ARGVS_)
-  return GTEST_CUSTOM_GET_ARGVS_();
-#else  // defined(GTEST_CUSTOM_GET_ARGVS_)
-  return g_argvs;
-#endif  // defined(GTEST_CUSTOM_GET_ARGVS_)
-}
-
-// Returns the current application's name, removing directory path if that
-// is present.
-FilePath GetCurrentExecutableName() {
-  FilePath result;
-
-#if GTEST_OS_WINDOWS
-  result.Set(FilePath(GetArgvs()[0]).RemoveExtension("exe"));
-#else
-  result.Set(FilePath(GetArgvs()[0]));
-#endif  // GTEST_OS_WINDOWS
-
-  return result.RemoveDirectoryName();
-}
-
-// Functions for processing the gtest_output flag.
-
-// Returns the output format, or "" for normal printed output.
-std::string UnitTestOptions::GetOutputFormat() {
-  const char* const gtest_output_flag = GTEST_FLAG(output).c_str();
-  if (gtest_output_flag == NULL) return std::string("");
-
-  const char* const colon = strchr(gtest_output_flag, ':');
-  return (colon == NULL) ?
-      std::string(gtest_output_flag) :
-      std::string(gtest_output_flag, colon - gtest_output_flag);
-}
-
-// Returns the name of the requested output file, or the default if none
-// was explicitly specified.
-std::string UnitTestOptions::GetAbsolutePathToOutputFile() {
-  const char* const gtest_output_flag = GTEST_FLAG(output).c_str();
-  if (gtest_output_flag == NULL)
-    return "";
-
-  const char* const colon = strchr(gtest_output_flag, ':');
-  if (colon == NULL)
-    return internal::FilePath::ConcatPaths(
-        internal::FilePath(
-            UnitTest::GetInstance()->original_working_dir()),
-        internal::FilePath(kDefaultOutputFile)).string();
-
-  internal::FilePath output_name(colon + 1);
-  if (!output_name.IsAbsolutePath())
-    // TODO(wan@google.com): on Windows \some\path is not an absolute
-    // path (as its meaning depends on the current drive), yet the
-    // following logic for turning it into an absolute path is wrong.
-    // Fix it.
-    output_name = internal::FilePath::ConcatPaths(
-        internal::FilePath(UnitTest::GetInstance()->original_working_dir()),
-        internal::FilePath(colon + 1));
-
-  if (!output_name.IsDirectory())
-    return output_name.string();
-
-  internal::FilePath result(internal::FilePath::GenerateUniqueFileName(
-      output_name, internal::GetCurrentExecutableName(),
-      GetOutputFormat().c_str()));
-  return result.string();
-}
-
-// Returns true iff the wildcard pattern matches the string.  The
-// first ':' or '\0' character in pattern marks the end of it.
-//
-// This recursive algorithm isn't very efficient, but is clear and
-// works well enough for matching test names, which are short.
-bool UnitTestOptions::PatternMatchesString(const char *pattern,
-                                           const char *str) {
-  switch (*pattern) {
-    case '\0':
-    case ':':  // Either ':' or '\0' marks the end of the pattern.
-      return *str == '\0';
-    case '?':  // Matches any single character.
-      return *str != '\0' && PatternMatchesString(pattern + 1, str + 1);
-    case '*':  // Matches any string (possibly empty) of characters.
-      return (*str != '\0' && PatternMatchesString(pattern, str + 1)) ||
-          PatternMatchesString(pattern + 1, str);
-    default:  // Non-special character.  Matches itself.
-      return *pattern == *str &&
-          PatternMatchesString(pattern + 1, str + 1);
-  }
-}
-
-bool UnitTestOptions::MatchesFilter(
-    const std::string& name, const char* filter) {
-  const char *cur_pattern = filter;
-  for (;;) {
-    if (PatternMatchesString(cur_pattern, name.c_str())) {
-      return true;
-    }
-
-    // Finds the next pattern in the filter.
-    cur_pattern = strchr(cur_pattern, ':');
-
-    // Returns if no more pattern can be found.
-    if (cur_pattern == NULL) {
-      return false;
-    }
-
-    // Skips the pattern separater (the ':' character).
-    cur_pattern++;
-  }
-}
-
-// Returns true iff the user-specified filter matches the test case
-// name and the test name.
-bool UnitTestOptions::FilterMatchesTest(const std::string &test_case_name,
-                                        const std::string &test_name) {
-  const std::string& full_name = test_case_name + "." + test_name.c_str();
-
-  // Split --gtest_filter at '-', if there is one, to separate into
-  // positive filter and negative filter portions
-  const char* const p = GTEST_FLAG(filter).c_str();
-  const char* const dash = strchr(p, '-');
-  std::string positive;
-  std::string negative;
-  if (dash == NULL) {
-    positive = GTEST_FLAG(filter).c_str();  // Whole string is a positive filter
-    negative = "";
-  } else {
-    positive = std::string(p, dash);   // Everything up to the dash
-    negative = std::string(dash + 1);  // Everything after the dash
-    if (positive.empty()) {
-      // Treat '-test1' as the same as '*-test1'
-      positive = kUniversalFilter;
-    }
-  }
-
-  // A filter is a colon-separated list of patterns.  It matches a
-  // test if any pattern in it matches the test.
-  return (MatchesFilter(full_name, positive.c_str()) &&
-          !MatchesFilter(full_name, negative.c_str()));
-}
-
-#if GTEST_HAS_SEH
-// Returns EXCEPTION_EXECUTE_HANDLER if Google Test should handle the
-// given SEH exception, or EXCEPTION_CONTINUE_SEARCH otherwise.
-// This function is useful as an __except condition.
-int UnitTestOptions::GTestShouldProcessSEH(DWORD exception_code) {
-  // Google Test should handle a SEH exception if:
-  //   1. the user wants it to, AND
-  //   2. this is not a breakpoint exception, AND
-  //   3. this is not a C++ exception (VC++ implements them via SEH,
-  //      apparently).
-  //
-  // SEH exception code for C++ exceptions.
-  // (see http://support.microsoft.com/kb/185294 for more information).
-  const DWORD kCxxExceptionCode = 0xe06d7363;
-
-  bool should_handle = true;
-
-  if (!GTEST_FLAG(catch_exceptions))
-    should_handle = false;
-  else if (exception_code == EXCEPTION_BREAKPOINT)
-    should_handle = false;
-  else if (exception_code == kCxxExceptionCode)
-    should_handle = false;
-
-  return should_handle ? EXCEPTION_EXECUTE_HANDLER : EXCEPTION_CONTINUE_SEARCH;
-}
-#endif  // GTEST_HAS_SEH
-
-}  // namespace internal
-
-// The c'tor sets this object as the test part result reporter used by
-// Google Test.  The 'result' parameter specifies where to report the
-// results. Intercepts only failures from the current thread.
-ScopedFakeTestPartResultReporter::ScopedFakeTestPartResultReporter(
-    TestPartResultArray* result)
-    : intercept_mode_(INTERCEPT_ONLY_CURRENT_THREAD),
-      result_(result) {
-  Init();
-}
-
-// The c'tor sets this object as the test part result reporter used by
-// Google Test.  The 'result' parameter specifies where to report the
-// results.
-ScopedFakeTestPartResultReporter::ScopedFakeTestPartResultReporter(
-    InterceptMode intercept_mode, TestPartResultArray* result)
-    : intercept_mode_(intercept_mode),
-      result_(result) {
-  Init();
-}
-
-void ScopedFakeTestPartResultReporter::Init() {
-  internal::UnitTestImpl* const impl = internal::GetUnitTestImpl();
-  if (intercept_mode_ == INTERCEPT_ALL_THREADS) {
-    old_reporter_ = impl->GetGlobalTestPartResultReporter();
-    impl->SetGlobalTestPartResultReporter(this);
-  } else {
-    old_reporter_ = impl->GetTestPartResultReporterForCurrentThread();
-    impl->SetTestPartResultReporterForCurrentThread(this);
-  }
-}
-
-// The d'tor restores the test part result reporter used by Google Test
-// before.
-ScopedFakeTestPartResultReporter::~ScopedFakeTestPartResultReporter() {
-  internal::UnitTestImpl* const impl = internal::GetUnitTestImpl();
-  if (intercept_mode_ == INTERCEPT_ALL_THREADS) {
-    impl->SetGlobalTestPartResultReporter(old_reporter_);
-  } else {
-    impl->SetTestPartResultReporterForCurrentThread(old_reporter_);
-  }
-}
-
-// Increments the test part result count and remembers the result.
-// This method is from the TestPartResultReporterInterface interface.
-void ScopedFakeTestPartResultReporter::ReportTestPartResult(
-    const TestPartResult& result) {
-  result_->Append(result);
-}
-
-namespace internal {
-
-// Returns the type ID of ::testing::Test.  We should always call this
-// instead of GetTypeId< ::testing::Test>() to get the type ID of
-// testing::Test.  This is to work around a suspected linker bug when
-// using Google Test as a framework on Mac OS X.  The bug causes
-// GetTypeId< ::testing::Test>() to return different values depending
-// on whether the call is from the Google Test framework itself or
-// from user test code.  GetTestTypeId() is guaranteed to always
-// return the same value, as it always calls GetTypeId<>() from the
-// gtest.cc, which is within the Google Test framework.
-TypeId GetTestTypeId() {
-  return GetTypeId<Test>();
-}
-
-// The value of GetTestTypeId() as seen from within the Google Test
-// library.  This is solely for testing GetTestTypeId().
-extern const TypeId kTestTypeIdInGoogleTest = GetTestTypeId();
-
-// This predicate-formatter checks that 'results' contains a test part
-// failure of the given type and that the failure message contains the
-// given substring.
-AssertionResult HasOneFailure(const char* /* results_expr */,
-                              const char* /* type_expr */,
-                              const char* /* substr_expr */,
-                              const TestPartResultArray& results,
-                              TestPartResult::Type type,
-                              const string& substr) {
-  const std::string expected(type == TestPartResult::kFatalFailure ?
-                        "1 fatal failure" :
-                        "1 non-fatal failure");
-  Message msg;
-  if (results.size() != 1) {
-    msg << "Expected: " << expected << "\n"
-        << "  Actual: " << results.size() << " failures";
-    for (int i = 0; i < results.size(); i++) {
-      msg << "\n" << results.GetTestPartResult(i);
-    }
-    return AssertionFailure() << msg;
-  }
-
-  const TestPartResult& r = results.GetTestPartResult(0);
-  if (r.type() != type) {
-    return AssertionFailure() << "Expected: " << expected << "\n"
-                              << "  Actual:\n"
-                              << r;
-  }
-
-  if (strstr(r.message(), substr.c_str()) == NULL) {
-    return AssertionFailure() << "Expected: " << expected << " containing \""
-                              << substr << "\"\n"
-                              << "  Actual:\n"
-                              << r;
-  }
-
-  return AssertionSuccess();
-}
-
-// The constructor of SingleFailureChecker remembers where to look up
-// test part results, what type of failure we expect, and what
-// substring the failure message should contain.
-SingleFailureChecker:: SingleFailureChecker(
-    const TestPartResultArray* results,
-    TestPartResult::Type type,
-    const string& substr)
-    : results_(results),
-      type_(type),
-      substr_(substr) {}
-
-// The destructor of SingleFailureChecker verifies that the given
-// TestPartResultArray contains exactly one failure that has the given
-// type and contains the given substring.  If that's not the case, a
-// non-fatal failure will be generated.
-SingleFailureChecker::~SingleFailureChecker() {
-  EXPECT_PRED_FORMAT3(HasOneFailure, *results_, type_, substr_);
-}
-
-DefaultGlobalTestPartResultReporter::DefaultGlobalTestPartResultReporter(
-    UnitTestImpl* unit_test) : unit_test_(unit_test) {}
-
-void DefaultGlobalTestPartResultReporter::ReportTestPartResult(
-    const TestPartResult& result) {
-  unit_test_->current_test_result()->AddTestPartResult(result);
-  unit_test_->listeners()->repeater()->OnTestPartResult(result);
-}
-
-DefaultPerThreadTestPartResultReporter::DefaultPerThreadTestPartResultReporter(
-    UnitTestImpl* unit_test) : unit_test_(unit_test) {}
-
-void DefaultPerThreadTestPartResultReporter::ReportTestPartResult(
-    const TestPartResult& result) {
-  unit_test_->GetGlobalTestPartResultReporter()->ReportTestPartResult(result);
-}
-
-// Returns the global test part result reporter.
-TestPartResultReporterInterface*
-UnitTestImpl::GetGlobalTestPartResultReporter() {
-  internal::MutexLock lock(&global_test_part_result_reporter_mutex_);
-  return global_test_part_result_repoter_;
-}
-
-// Sets the global test part result reporter.
-void UnitTestImpl::SetGlobalTestPartResultReporter(
-    TestPartResultReporterInterface* reporter) {
-  internal::MutexLock lock(&global_test_part_result_reporter_mutex_);
-  global_test_part_result_repoter_ = reporter;
-}
-
-// Returns the test part result reporter for the current thread.
-TestPartResultReporterInterface*
-UnitTestImpl::GetTestPartResultReporterForCurrentThread() {
-  return per_thread_test_part_result_reporter_.get();
-}
-
-// Sets the test part result reporter for the current thread.
-void UnitTestImpl::SetTestPartResultReporterForCurrentThread(
-    TestPartResultReporterInterface* reporter) {
-  per_thread_test_part_result_reporter_.set(reporter);
-}
-
-// Gets the number of successful test cases.
-int UnitTestImpl::successful_test_case_count() const {
-  return CountIf(test_cases_, TestCasePassed);
-}
-
-// Gets the number of failed test cases.
-int UnitTestImpl::failed_test_case_count() const {
-  return CountIf(test_cases_, TestCaseFailed);
-}
-
-// Gets the number of all test cases.
-int UnitTestImpl::total_test_case_count() const {
-  return static_cast<int>(test_cases_.size());
-}
-
-// Gets the number of all test cases that contain at least one test
-// that should run.
-int UnitTestImpl::test_case_to_run_count() const {
-  return CountIf(test_cases_, ShouldRunTestCase);
-}
-
-// Gets the number of successful tests.
-int UnitTestImpl::successful_test_count() const {
-  return SumOverTestCaseList(test_cases_, &TestCase::successful_test_count);
-}
-
-// Gets the number of failed tests.
-int UnitTestImpl::failed_test_count() const {
-  return SumOverTestCaseList(test_cases_, &TestCase::failed_test_count);
-}
-
-// Gets the number of disabled tests that will be reported in the XML report.
-int UnitTestImpl::reportable_disabled_test_count() const {
-  return SumOverTestCaseList(test_cases_,
-                             &TestCase::reportable_disabled_test_count);
-}
-
-// Gets the number of disabled tests.
-int UnitTestImpl::disabled_test_count() const {
-  return SumOverTestCaseList(test_cases_, &TestCase::disabled_test_count);
-}
-
-// Gets the number of tests to be printed in the XML report.
-int UnitTestImpl::reportable_test_count() const {
-  return SumOverTestCaseList(test_cases_, &TestCase::reportable_test_count);
-}
-
-// Gets the number of all tests.
-int UnitTestImpl::total_test_count() const {
-  return SumOverTestCaseList(test_cases_, &TestCase::total_test_count);
-}
-
-// Gets the number of tests that should run.
-int UnitTestImpl::test_to_run_count() const {
-  return SumOverTestCaseList(test_cases_, &TestCase::test_to_run_count);
-}
-
-// Returns the current OS stack trace as an std::string.
-//
-// The maximum number of stack frames to be included is specified by
-// the gtest_stack_trace_depth flag.  The skip_count parameter
-// specifies the number of top frames to be skipped, which doesn't
-// count against the number of frames to be included.
-//
-// For example, if Foo() calls Bar(), which in turn calls
-// CurrentOsStackTraceExceptTop(1), Foo() will be included in the
-// trace but Bar() and CurrentOsStackTraceExceptTop() won't.
-std::string UnitTestImpl::CurrentOsStackTraceExceptTop(int skip_count) {
-  return os_stack_trace_getter()->CurrentStackTrace(
-      static_cast<int>(GTEST_FLAG(stack_trace_depth)),
-      skip_count + 1
-      // Skips the user-specified number of frames plus this function
-      // itself.
-      );  // NOLINT
-}
-
-// Returns the current time in milliseconds.
-TimeInMillis GetTimeInMillis() {
-#if GTEST_OS_WINDOWS_MOBILE || defined(__BORLANDC__)
-  // Difference between 1970-01-01 and 1601-01-01 in milliseconds.
-  // http://analogous.blogspot.com/2005/04/epoch.html
-  const TimeInMillis kJavaEpochToWinFileTimeDelta =
-    static_cast<TimeInMillis>(116444736UL) * 100000UL;
-  const DWORD kTenthMicrosInMilliSecond = 10000;
-
-  SYSTEMTIME now_systime;
-  FILETIME now_filetime;
-  ULARGE_INTEGER now_int64;
-  // TODO(kenton@google.com): Shouldn't this just use
-  //   GetSystemTimeAsFileTime()?
-  GetSystemTime(&now_systime);
-  if

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/50] [abbrv] hadoop git commit: HDFS-11094. Send back HAState along with NamespaceInfo during a versionRequest as an optional parameter. Contributed by Eric Badger

Posted by st...@apache.org.
HDFS-11094. Send back HAState along with NamespaceInfo during a versionRequest as an optional parameter. Contributed by Eric Badger


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

Branch: refs/heads/HADOOP-13345
Commit: 8c4680852b20ad0e65e77dd123c9ba5bb6f2fa39
Parents: 43ebff2
Author: Mingliang Liu <li...@apache.org>
Authored: Mon Dec 5 11:34:13 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Mon Dec 5 11:48:58 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 76 +++++++++++++-------
 .../hdfs/server/datanode/BPOfferService.java    | 10 ++-
 .../hdfs/server/datanode/BPServiceActor.java    |  4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  8 ++-
 .../hdfs/server/protocol/NamespaceInfo.java     | 26 +++++++
 .../hadoop-hdfs/src/main/proto/HdfsServer.proto |  2 +
 .../server/datanode/TestBPOfferService.java     | 31 ++++++++
 .../hdfs/server/namenode/TestFSNamesystem.java  | 21 ++++++
 8 files changed, 148 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 78371f5..1e6d882 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -26,7 +26,7 @@ import com.google.protobuf.ByteString;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -338,7 +338,8 @@ public class PBHelper {
     StorageInfoProto storage = info.getStorageInfo();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
         info.getBlockPoolID(), storage.getCTime(), info.getBuildVersion(),
-        info.getSoftwareVersion(), info.getCapabilities());
+        info.getSoftwareVersion(), info.getCapabilities(),
+        convert(info.getState()));
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -744,43 +745,68 @@ public class PBHelper {
   }
   
   public static NamespaceInfoProto convert(NamespaceInfo info) {
-    return NamespaceInfoProto.newBuilder()
-        .setBlockPoolID(info.getBlockPoolID())
+    NamespaceInfoProto.Builder builder = NamespaceInfoProto.newBuilder();
+    builder.setBlockPoolID(info.getBlockPoolID())
         .setBuildVersion(info.getBuildVersion())
         .setUnused(0)
         .setStorageInfo(PBHelper.convert((StorageInfo)info))
         .setSoftwareVersion(info.getSoftwareVersion())
-        .setCapabilities(info.getCapabilities())
-        .build();
+        .setCapabilities(info.getCapabilities());
+    HAServiceState state = info.getState();
+    if(state != null) {
+      builder.setState(convert(info.getState()));
+    }
+    return builder.build();
   }
 
-  public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
-    if (s == null) return null;
-    switch (s.getState()) {
+  public static HAServiceState convert(HAServiceStateProto s) {
+    if (s == null) {
+      return null;
+    }
+    switch (s) {
+    case INITIALIZING:
+      return HAServiceState.INITIALIZING;
     case ACTIVE:
-      return new NNHAStatusHeartbeat(HAServiceState.ACTIVE, s.getTxid());
+      return HAServiceState.ACTIVE;
     case STANDBY:
-      return new NNHAStatusHeartbeat(HAServiceState.STANDBY, s.getTxid());
+      return HAServiceState.STANDBY;
     default:
-      throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" + s.getState());
+      throw new IllegalArgumentException("Unexpected HAServiceStateProto:"
+          + s);
     }
   }
 
+  public static HAServiceStateProto convert(HAServiceState s) {
+    if (s == null) {
+      return null;
+    }
+    switch (s) {
+    case INITIALIZING:
+      return HAServiceStateProto.INITIALIZING;
+    case ACTIVE:
+      return HAServiceStateProto.ACTIVE;
+    case STANDBY:
+      return HAServiceStateProto.STANDBY;
+    default:
+      throw new IllegalArgumentException("Unexpected HAServiceState:"
+          + s);
+    }
+  }
+
+  public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
+    if (s == null) {
+      return null;
+    }
+    return new NNHAStatusHeartbeat(convert(s.getState()), s.getTxid());
+  }
+
   public static NNHAStatusHeartbeatProto convert(NNHAStatusHeartbeat hb) {
-    if (hb == null) return null;
-    NNHAStatusHeartbeatProto.Builder builder =
-      NNHAStatusHeartbeatProto.newBuilder();
-    switch (hb.getState()) {
-      case ACTIVE:
-        builder.setState(HAServiceProtocolProtos.HAServiceStateProto.ACTIVE);
-        break;
-      case STANDBY:
-        builder.setState(HAServiceProtocolProtos.HAServiceStateProto.STANDBY);
-        break;
-      default:
-        throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" +
-            hb.getState());
+    if (hb == null) {
+      return null;
     }
+    NNHAStatusHeartbeatProto.Builder builder =
+        NNHAStatusHeartbeatProto.newBuilder();
+    builder.setState(convert(hb.getState()));
     builder.setTxid(hb.getTxId());
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 00102eb..00e6b3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -307,8 +307,16 @@ class BPOfferService {
    * verifies that this namespace matches (eg to prevent a misconfiguration
    * where a StandbyNode from a different cluster is specified)
    */
-  void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
+  void verifyAndSetNamespaceInfo(BPServiceActor actor, NamespaceInfo nsInfo)
+    throws IOException {
     writeLock();
+
+    if(nsInfo.getState() == HAServiceState.ACTIVE
+        && bpServiceToActive == null) {
+      LOG.info("Acknowledging ACTIVE Namenode during handshake" + actor);
+      bpServiceToActive = actor;
+    }
+
     try {
       if (this.bpNSInfo == null) {
         this.bpNSInfo = nsInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index f3247fc..dffe14f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -269,11 +269,11 @@ class BPServiceActor implements Runnable {
     // First phase of the handshake with NN - get the namespace
     // info.
     NamespaceInfo nsInfo = retrieveNamespaceInfo();
-    
+
     // Verify that this matches the other NN in this HA pair.
     // This also initializes our block pool in the DN if we are
     // the first NN connection for this BP.
-    bpos.verifyAndSetNamespaceInfo(nsInfo);
+    bpos.verifyAndSetNamespaceInfo(this, nsInfo);
     
     // Second phase of the handshake with the NN.
     register(nsInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 8a750a0..90fb924 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1594,7 +1594,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   NamespaceInfo unprotectedGetNamespaceInfo() {
     return new NamespaceInfo(getFSImage().getStorage().getNamespaceID(),
         getClusterId(), getBlockPoolId(),
-        getFSImage().getStorage().getCTime());
+        getFSImage().getStorage().getCTime(), getState());
   }
 
   /**
@@ -4531,12 +4531,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       return 0;
     }
   }
-  
+
   @Metric
   public int getBlockCapacity() {
     return blockManager.getCapacity();
   }
 
+  public HAServiceState getState() {
+    return haContext == null ? null : haContext.getState().getServiceState();
+  }
+
   @Override // FSNamesystemMBean
   public String getFSState() {
     return isInSafeMode() ? "safeMode" : "Operational";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
index 90d0aac..66ce9ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -44,6 +45,7 @@ public class NamespaceInfo extends StorageInfo {
   String blockPoolID = "";    // id of the block pool
   String softwareVersion;
   long capabilities;
+  HAServiceState state;
 
   // only authoritative on the server-side to determine advertisement to
   // clients.  enum will update the supported values
@@ -88,6 +90,14 @@ public class NamespaceInfo extends StorageInfo {
         CAPABILITIES_SUPPORTED);
   }
 
+  public NamespaceInfo(int nsID, String clusterID, String bpID,
+      long cT, String buildVersion, String softwareVersion,
+      long capabilities, HAServiceState st) {
+    this(nsID, clusterID, bpID, cT, buildVersion, softwareVersion,
+        capabilities);
+    this.state = st;
+  }
+
   // for use by server and/or client
   public NamespaceInfo(int nsID, String clusterID, String bpID,
       long cT, String buildVersion, String softwareVersion,
@@ -105,6 +115,13 @@ public class NamespaceInfo extends StorageInfo {
     this(nsID, clusterID, bpID, cT, Storage.getBuildVersion(),
         VersionInfo.getVersion());
   }
+
+  public NamespaceInfo(int nsID, String clusterID, String bpID,
+      long cT, HAServiceState st) {
+    this(nsID, clusterID, bpID, cT, Storage.getBuildVersion(),
+        VersionInfo.getVersion());
+    this.state = st;
+  }
   
   public long getCapabilities() {
     return capabilities;
@@ -115,6 +132,11 @@ public class NamespaceInfo extends StorageInfo {
     this.capabilities = capabilities;
   }
 
+  @VisibleForTesting
+  public void setState(HAServiceState state) {
+    this.state = state;
+  }
+
   public boolean isCapabilitySupported(Capability capability) {
     Preconditions.checkArgument(capability != Capability.UNKNOWN,
         "cannot test for unknown capability");
@@ -134,6 +156,10 @@ public class NamespaceInfo extends StorageInfo {
     return softwareVersion;
   }
 
+  public HAServiceState getState() {
+    return state;
+  }
+
   @Override
   public String toString(){
     return super.toString() + ";bpid=" + blockPoolID;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
index 910e03b..d7deebf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
@@ -32,6 +32,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
 import "hdfs.proto";
+import "HAServiceProtocol.proto";
 
 /**
  * Block access token information
@@ -101,6 +102,7 @@ message NamespaceInfoProto {
   required StorageInfoProto storageInfo = 4;// Node information
   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
   optional uint64 capabilities = 6 [default = 0]; // feature flags
+  optional hadoop.common.HAServiceStateProto state = 7;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index 2d50c75..f8f0a3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
@@ -799,4 +800,34 @@ public class TestBPOfferService {
     }
     return -1;
   }
+
+   /*
+    *
+    */
+  @Test
+  public void testNNHAStateUpdateFromVersionRequest() throws Exception {
+    final BPOfferService bpos = setupBPOSForNNs(mockNN1, mockNN2);
+    BPServiceActor actor = bpos.getBPServiceActors().get(0);
+    bpos.start();
+    waitForInitialization(bpos);
+    // Should start with neither NN as active.
+    assertNull(bpos.getActiveNN());
+
+    // getNamespaceInfo() will not include HAServiceState
+    NamespaceInfo nsInfo = mockNN1.versionRequest();
+    bpos.verifyAndSetNamespaceInfo(actor, nsInfo);
+
+    assertNull(bpos.getActiveNN());
+
+    // Change mock so getNamespaceInfo() will include HAServiceState
+    Mockito.doReturn(new NamespaceInfo(1, FAKE_CLUSTERID, FAKE_BPID, 0,
+        HAServiceState.ACTIVE)).when(mockNN1).versionRequest();
+
+    // Update the bpos NamespaceInfo
+    nsInfo = mockNN1.versionRequest();
+    bpos.verifyAndSetNamespaceInfo(actor, nsInfo);
+
+    assertNotNull(bpos.getActiveNN());
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c468085/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
index f02c679..6a0dd6f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
@@ -33,6 +33,7 @@ import java.util.Collection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.junit.After;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -155,6 +157,25 @@ public class TestFSNamesystem {
   }
 
   @Test
+  public void testHAStateInNamespaceInfo() throws IOException {
+    Configuration conf = new Configuration();
+
+    FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
+    FSImage fsImage = Mockito.mock(FSImage.class);
+    Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
+    NNStorage nnStorage = Mockito.mock(NNStorage.class);
+    Mockito.when(fsImage.getStorage()).thenReturn(nnStorage);
+
+    FSNamesystem fsNamesystem = new FSNamesystem(conf, fsImage);
+    FSNamesystem fsn = Mockito.spy(fsNamesystem);
+    Mockito.when(fsn.getState()).thenReturn(
+        HAServiceProtocol.HAServiceState.ACTIVE);
+
+    NamespaceInfo nsInfo = fsn.unprotectedGetNamespaceInfo();
+    assertNotNull(nsInfo.getState());
+  }
+
+  @Test
   public void testReset() throws Exception {
     Configuration conf = new Configuration();
     FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/50] [abbrv] hadoop git commit: Revert "HADOOP-10930. Refactor: Wrap Datanode IO related operations. Contributed by Xiaoyu Yao."

Posted by st...@apache.org.
Revert "HADOOP-10930. Refactor: Wrap Datanode IO related operations. Contributed by Xiaoyu Yao."

This reverts commit aeecfa24f4fb6af289920cbf8830c394e66bd78e.


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

Branch: refs/heads/HADOOP-13345
Commit: dcedb72af468128458e597f08d22f5c34b744ae5
Parents: 15dd1f3
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Mon Dec 5 12:08:48 2016 -0800
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Mon Dec 5 12:44:20 2016 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockReceiver.java     |  66 ++++---
 .../hdfs/server/datanode/BlockSender.java       | 105 +++++++----
 .../hadoop/hdfs/server/datanode/DNConf.java     |   4 -
 .../hdfs/server/datanode/DataStorage.java       |   5 -
 .../hdfs/server/datanode/LocalReplica.java      | 179 ++++++-------------
 .../server/datanode/LocalReplicaInPipeline.java |  30 ++--
 .../hdfs/server/datanode/ReplicaInPipeline.java |   4 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   3 +-
 .../datanode/fsdataset/ReplicaInputStreams.java | 102 +----------
 .../fsdataset/ReplicaOutputStreams.java         | 107 +----------
 .../datanode/fsdataset/impl/BlockPoolSlice.java |  32 ++--
 .../impl/FsDatasetAsyncDiskService.java         |   7 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   5 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   5 +-
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |   2 +-
 .../server/datanode/SimulatedFSDataset.java     |  13 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   2 +-
 .../server/datanode/TestSimulatedFSDataset.java |   2 +-
 .../extdataset/ExternalDatasetImpl.java         |   4 +-
 .../extdataset/ExternalReplicaInPipeline.java   |   6 +-
 20 files changed, 238 insertions(+), 445 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index f372072..39419c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -24,7 +24,10 @@ import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
+import java.io.FileDescriptor;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.nio.ByteBuffer;
@@ -50,6 +53,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
@@ -84,6 +88,8 @@ class BlockReceiver implements Closeable {
    * the DataNode needs to recalculate checksums before writing.
    */
   private final boolean needsChecksumTranslation;
+  private OutputStream out = null; // to block file at local disk
+  private FileDescriptor outFd;
   private DataOutputStream checksumOut = null; // to crc file at local disk
   private final int bytesPerChecksum;
   private final int checksumSize;
@@ -244,8 +250,7 @@ class BlockReceiver implements Closeable {
       
       final boolean isCreate = isDatanode || isTransfer 
           || stage == BlockConstructionStage.PIPELINE_SETUP_CREATE;
-      streams = replicaInfo.createStreams(isCreate, requestedChecksum,
-          datanodeSlowLogThresholdMs);
+      streams = replicaInfo.createStreams(isCreate, requestedChecksum);
       assert streams != null : "null streams!";
 
       // read checksum meta information
@@ -255,6 +260,13 @@ class BlockReceiver implements Closeable {
       this.bytesPerChecksum = diskChecksum.getBytesPerChecksum();
       this.checksumSize = diskChecksum.getChecksumSize();
 
+      this.out = streams.getDataOut();
+      if (out instanceof FileOutputStream) {
+        this.outFd = ((FileOutputStream)out).getFD();
+      } else {
+        LOG.warn("Could not get file descriptor for outputstream of class " +
+            out.getClass());
+      }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
           streams.getChecksumOut(), DFSUtilClient.getSmallBufferSize(
           datanode.getConf())));
@@ -307,7 +319,7 @@ class BlockReceiver implements Closeable {
     packetReceiver.close();
 
     IOException ioe = null;
-    if (syncOnClose && (streams.getDataOut() != null || checksumOut != null)) {
+    if (syncOnClose && (out != null || checksumOut != null)) {
       datanode.metrics.incrFsyncCount();      
     }
     long flushTotalNanos = 0;
@@ -336,9 +348,9 @@ class BlockReceiver implements Closeable {
     }
     // close block file
     try {
-      if (streams.getDataOut() != null) {
+      if (out != null) {
         long flushStartNanos = System.nanoTime();
-        streams.flushDataOut();
+        out.flush();
         long flushEndNanos = System.nanoTime();
         if (syncOnClose) {
           long fsyncStartNanos = flushEndNanos;
@@ -347,13 +359,14 @@ class BlockReceiver implements Closeable {
         }
         flushTotalNanos += flushEndNanos - flushStartNanos;
         measuredFlushTime = true;
-        streams.closeDataStream();
+        out.close();
+        out = null;
       }
     } catch (IOException e) {
       ioe = e;
     }
     finally{
-      streams.close();
+      IOUtils.closeStream(out);
     }
     if (replicaHandler != null) {
       IOUtils.cleanup(null, replicaHandler);
@@ -406,9 +419,9 @@ class BlockReceiver implements Closeable {
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
-    if (streams.getDataOut() != null) {
+    if (out != null) {
       long flushStartNanos = System.nanoTime();
-      streams.flushDataOut();
+      out.flush();
       long flushEndNanos = System.nanoTime();
       if (isSync) {
         long fsyncStartNanos = flushEndNanos;
@@ -417,10 +430,10 @@ class BlockReceiver implements Closeable {
       }
       flushTotalNanos += flushEndNanos - flushStartNanos;
     }
-    if (checksumOut != null || streams.getDataOut() != null) {
+    if (checksumOut != null || out != null) {
       datanode.metrics.addFlushNanos(flushTotalNanos);
       if (isSync) {
-        datanode.metrics.incrFsyncCount();
+    	  datanode.metrics.incrFsyncCount();      
       }
     }
     long duration = Time.monotonicNow() - begin;
@@ -703,12 +716,16 @@ class BlockReceiver implements Closeable {
           int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
           
           // Write data to disk.
-          long duration = streams.writeToDisk(dataBuf.array(),
-              startByteToDisk, numBytesToDisk);
-
+          long begin = Time.monotonicNow();
+          out.write(dataBuf.array(), startByteToDisk, numBytesToDisk);
+          long duration = Time.monotonicNow() - begin;
           if (duration > maxWriteToDiskMs) {
             maxWriteToDiskMs = duration;
           }
+          if (duration > datanodeSlowLogThresholdMs) {
+            LOG.warn("Slow BlockReceiver write data to disk cost:" + duration
+                + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
+          }
 
           final byte[] lastCrc;
           if (shouldNotWriteChecksum) {
@@ -825,7 +842,7 @@ class BlockReceiver implements Closeable {
 
   private void manageWriterOsCache(long offsetInBlock) {
     try {
-      if (streams.getOutFd() != null &&
+      if (outFd != null &&
           offsetInBlock > lastCacheManagementOffset + CACHE_DROP_LAG_BYTES) {
         long begin = Time.monotonicNow();
         //
@@ -840,11 +857,12 @@ class BlockReceiver implements Closeable {
         if (syncBehindWrites) {
           if (syncBehindWritesInBackground) {
             this.datanode.getFSDataset().submitBackgroundSyncFileRangeRequest(
-                block, streams, lastCacheManagementOffset,
+                block, outFd, lastCacheManagementOffset,
                 offsetInBlock - lastCacheManagementOffset,
                 SYNC_FILE_RANGE_WRITE);
           } else {
-            streams.syncFileRangeIfPossible(lastCacheManagementOffset,
+            NativeIO.POSIX.syncFileRangeIfPossible(outFd,
+                lastCacheManagementOffset,
                 offsetInBlock - lastCacheManagementOffset,
                 SYNC_FILE_RANGE_WRITE);
           }
@@ -861,8 +879,8 @@ class BlockReceiver implements Closeable {
         //                     
         long dropPos = lastCacheManagementOffset - CACHE_DROP_LAG_BYTES;
         if (dropPos > 0 && dropCacheBehindWrites) {
-          streams.dropCacheBehindWrites(block.getBlockName(), 0, dropPos,
-              POSIX_FADV_DONTNEED);
+          NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+              block.getBlockName(), outFd, 0, dropPos, POSIX_FADV_DONTNEED);
         }
         lastCacheManagementOffset = offsetInBlock;
         long duration = Time.monotonicNow() - begin;
@@ -971,7 +989,7 @@ class BlockReceiver implements Closeable {
               // The worst case is not recovering this RBW replica. 
               // Client will fall back to regular pipeline recovery.
             } finally {
-              IOUtils.closeStream(streams.getDataOut());
+              IOUtils.closeStream(out);
             }
             try {              
               // Even if the connection is closed after the ack packet is
@@ -1029,8 +1047,8 @@ class BlockReceiver implements Closeable {
    * will be overwritten.
    */
   private void adjustCrcFilePosition() throws IOException {
-    if (streams.getDataOut() != null) {
-      streams.flushDataOut();
+    if (out != null) {
+     out.flush();
     }
     if (checksumOut != null) {
       checksumOut.flush();
@@ -1076,10 +1094,10 @@ class BlockReceiver implements Closeable {
     byte[] crcbuf = new byte[checksumSize];
     try (ReplicaInputStreams instr =
         datanode.data.getTmpInputStreams(block, blkoff, ckoff)) {
-      instr.readDataFully(buf, 0, sizePartialChunk);
+      IOUtils.readFully(instr.getDataIn(), buf, 0, sizePartialChunk);
 
       // open meta file and read in crc value computer earlier
-      instr.readChecksumFully(crcbuf, 0, crcbuf.length);
+      IOUtils.readFully(instr.getChecksumIn(), crcbuf, 0, crcbuf.length);
     }
 
     // compute crc of partial chunk from data read in the block file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index 9182c88..a1b1f86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -41,11 +42,11 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
@@ -119,11 +120,12 @@ class BlockSender implements java.io.Closeable {
   
   /** the block to read from */
   private final ExtendedBlock block;
-
-  /** InputStreams and file descriptors to read block/checksum. */
-  private ReplicaInputStreams ris;
+  /** Stream to read block data from */
+  private InputStream blockIn;
   /** updated while using transferTo() */
   private long blockInPosition = -1;
+  /** Stream to read checksum */
+  private DataInputStream checksumIn;
   /** Checksum utility */
   private final DataChecksum checksum;
   /** Initial position to read */
@@ -150,6 +152,11 @@ class BlockSender implements java.io.Closeable {
   private final String clientTraceFmt;
   private volatile ChunkChecksum lastChunkChecksum = null;
   private DataNode datanode;
+  
+  /** The file descriptor of the block being sent */
+  private FileDescriptor blockInFd;
+  /** The reference to the volume where the block is located */
+  private FsVolumeReference volumeRef;
 
   /** The replica of the block that is being read. */
   private final Replica replica;
@@ -194,9 +201,6 @@ class BlockSender implements java.io.Closeable {
               boolean sendChecksum, DataNode datanode, String clientTraceFmt,
               CachingStrategy cachingStrategy)
       throws IOException {
-    InputStream blockIn = null;
-    DataInputStream checksumIn = null;
-    FsVolumeReference volumeRef = null;
     try {
       this.block = block;
       this.corruptChecksumOk = corruptChecksumOk;
@@ -277,7 +281,7 @@ class BlockSender implements java.io.Closeable {
         (!is32Bit || length <= Integer.MAX_VALUE);
 
       // Obtain a reference before reading data
-      volumeRef = datanode.data.getVolume(block).obtainReference();
+      this.volumeRef = datanode.data.getVolume(block).obtainReference();
 
       /* 
        * (corruptChecksumOK, meta_file_exist): operation
@@ -401,9 +405,14 @@ class BlockSender implements java.io.Closeable {
         DataNode.LOG.debug("replica=" + replica);
       }
       blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
-      ris = new ReplicaInputStreams(blockIn, checksumIn, volumeRef);
+      if (blockIn instanceof FileInputStream) {
+        blockInFd = ((FileInputStream)blockIn).getFD();
+      } else {
+        blockInFd = null;
+      }
     } catch (IOException ioe) {
       IOUtils.closeStream(this);
+      IOUtils.closeStream(blockIn);
       throw ioe;
     }
   }
@@ -413,11 +422,12 @@ class BlockSender implements java.io.Closeable {
    */
   @Override
   public void close() throws IOException {
-    if (ris.getDataInFd() != null &&
+    if (blockInFd != null &&
         ((dropCacheBehindAllReads) ||
          (dropCacheBehindLargeReads && isLongRead()))) {
       try {
-        ris.dropCacheBehindReads(block.getBlockName(), lastCacheDropOffset,
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+            block.getBlockName(), blockInFd, lastCacheDropOffset,
             offset - lastCacheDropOffset, POSIX_FADV_DONTNEED);
       } catch (Exception e) {
         LOG.warn("Unable to drop cache on file close", e);
@@ -426,12 +436,32 @@ class BlockSender implements java.io.Closeable {
     if (curReadahead != null) {
       curReadahead.cancel();
     }
-
-    try {
-      ris.closeStreams();
-    } finally {
-      IOUtils.closeStream(ris);
-      ris = null;
+    
+    IOException ioe = null;
+    if(checksumIn!=null) {
+      try {
+        checksumIn.close(); // close checksum file
+      } catch (IOException e) {
+        ioe = e;
+      }
+      checksumIn = null;
+    }   
+    if(blockIn!=null) {
+      try {
+        blockIn.close(); // close data file
+      } catch (IOException e) {
+        ioe = e;
+      }
+      blockIn = null;
+      blockInFd = null;
+    }
+    if (volumeRef != null) {
+      IOUtils.cleanup(null, volumeRef);
+      volumeRef = null;
+    }
+    // throw IOException if there is any
+    if(ioe!= null) {
+      throw ioe;
     }
   }
   
@@ -535,7 +565,7 @@ class BlockSender implements java.io.Closeable {
     int checksumOff = pkt.position();
     byte[] buf = pkt.array();
     
-    if (checksumSize > 0 && ris.getChecksumIn() != null) {
+    if (checksumSize > 0 && checksumIn != null) {
       readChecksum(buf, checksumOff, checksumDataLen);
 
       // write in progress that we need to use to get last checksum
@@ -551,7 +581,7 @@ class BlockSender implements java.io.Closeable {
     
     int dataOff = checksumOff + checksumDataLen;
     if (!transferTo) { // normal transfer
-      ris.readDataFully(buf, dataOff, dataLen);
+      IOUtils.readFully(blockIn, buf, dataOff, dataLen);
 
       if (verifyChecksum) {
         verifyChecksum(buf, dataOff, dataLen, numChunks, checksumOff);
@@ -563,12 +593,12 @@ class BlockSender implements java.io.Closeable {
         SocketOutputStream sockOut = (SocketOutputStream)out;
         // First write header and checksums
         sockOut.write(buf, headerOff, dataOff - headerOff);
-
+        
         // no need to flush since we know out is not a buffered stream
-        FileChannel fileCh = ((FileInputStream)ris.getDataIn()).getChannel();
+        FileChannel fileCh = ((FileInputStream)blockIn).getChannel();
         LongWritable waitTime = new LongWritable();
         LongWritable transferTime = new LongWritable();
-        sockOut.transferToFully(fileCh, blockInPosition, dataLen,
+        sockOut.transferToFully(fileCh, blockInPosition, dataLen, 
             waitTime, transferTime);
         datanode.metrics.addSendDataPacketBlockedOnNetworkNanos(waitTime.get());
         datanode.metrics.addSendDataPacketTransferNanos(transferTime.get());
@@ -600,7 +630,7 @@ class BlockSender implements java.io.Closeable {
         if (!ioem.startsWith("Broken pipe") && !ioem.startsWith("Connection reset")) {
           LOG.error("BlockSender.sendChunks() exception: ", e);
           datanode.getBlockScanner().markSuspectBlock(
-              ris.getVolumeRef().getVolume().getStorageID(),
+              volumeRef.getVolume().getStorageID(),
               block);
         }
       }
@@ -623,15 +653,16 @@ class BlockSender implements java.io.Closeable {
    */
   private void readChecksum(byte[] buf, final int checksumOffset,
       final int checksumLen) throws IOException {
-    if (checksumSize <= 0 && ris.getChecksumIn() == null) {
+    if (checksumSize <= 0 && checksumIn == null) {
       return;
     }
     try {
-      ris.readChecksumFully(buf, checksumOffset, checksumLen);
+      checksumIn.readFully(buf, checksumOffset, checksumLen);
     } catch (IOException e) {
       LOG.warn(" Could not read or failed to verify checksum for data"
           + " at offset " + offset + " for block " + block, e);
-      ris.closeChecksumStream();
+      IOUtils.closeStream(checksumIn);
+      checksumIn = null;
       if (corruptChecksumOk) {
         if (checksumOffset < checksumLen) {
           // Just fill the array with zeros.
@@ -715,10 +746,10 @@ class BlockSender implements java.io.Closeable {
     
     lastCacheDropOffset = initialOffset;
 
-    if (isLongRead() && ris.getDataInFd() != null) {
+    if (isLongRead() && blockInFd != null) {
       // Advise that this file descriptor will be accessed sequentially.
-      ris.dropCacheBehindReads(block.getBlockName(), 0, 0,
-          POSIX_FADV_SEQUENTIAL);
+      NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+          block.getBlockName(), blockInFd, 0, 0, POSIX_FADV_SEQUENTIAL);
     }
     
     // Trigger readahead of beginning of file if configured.
@@ -730,10 +761,9 @@ class BlockSender implements java.io.Closeable {
       int pktBufSize = PacketHeader.PKT_MAX_HEADER_LEN;
       boolean transferTo = transferToAllowed && !verifyChecksum
           && baseStream instanceof SocketOutputStream
-          && ris.getDataIn() instanceof FileInputStream;
+          && blockIn instanceof FileInputStream;
       if (transferTo) {
-        FileChannel fileChannel =
-            ((FileInputStream)ris.getDataIn()).getChannel();
+        FileChannel fileChannel = ((FileInputStream)blockIn).getChannel();
         blockInPosition = fileChannel.position();
         streamForSendChunks = baseStream;
         maxChunksPerPacket = numberOfChunks(TRANSFERTO_BUFFER_SIZE);
@@ -788,16 +818,14 @@ class BlockSender implements java.io.Closeable {
   private void manageOsCache() throws IOException {
     // We can't manage the cache for this block if we don't have a file
     // descriptor to work with.
-    if (ris.getDataInFd() == null) {
-      return;
-    }
+    if (blockInFd == null) return;
 
     // Perform readahead if necessary
     if ((readaheadLength > 0) && (datanode.readaheadPool != null) &&
           (alwaysReadahead || isLongRead())) {
       curReadahead = datanode.readaheadPool.readaheadStream(
-          clientTraceFmt, ris.getDataInFd(), offset, readaheadLength,
-          Long.MAX_VALUE, curReadahead);
+          clientTraceFmt, blockInFd, offset, readaheadLength, Long.MAX_VALUE,
+          curReadahead);
     }
 
     // Drop what we've just read from cache, since we aren't
@@ -807,7 +835,8 @@ class BlockSender implements java.io.Closeable {
       long nextCacheDropOffset = lastCacheDropOffset + CACHE_DROP_INTERVAL_BYTES;
       if (offset >= nextCacheDropOffset) {
         long dropLength = offset - lastCacheDropOffset;
-        ris.dropCacheBehindReads(block.getBlockName(), lastCacheDropOffset,
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
+            block.getBlockName(), blockInFd, lastCacheDropOffset,
             dropLength, POSIX_FADV_DONTNEED);
         lastCacheDropOffset = offset;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index c1487b1..823d05c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -402,10 +402,6 @@ public class DNConf {
     return volsConfigured;
   }
 
-  public long getSlowIoWarningThresholdMs() {
-    return datanodeSlowIoWarningThresholdMs;
-  }
-
   int getMaxDataLength() {
     return maxDataLength;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index f4deb6d..29b14e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -1355,9 +1355,4 @@ public class DataStorage extends Storage {
   synchronized void removeBlockPoolStorage(String bpId) {
     bpStorageMap.remove(bpId);
   }
-
-  public static boolean fullyDelete(final File dir) {
-    boolean result = FileUtil.fullyDelete(dir);
-    return result;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
index 3615cd1..f829111 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
@@ -29,6 +29,9 @@ import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -43,8 +46,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -68,7 +69,15 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   private static final Map<String, File> internedBaseDirs = new HashMap<String, File>();
 
-  static final Logger LOG = LoggerFactory.getLogger(LocalReplica.class);
+  static final Log LOG = LogFactory.getLog(LocalReplica.class);
+  private final static boolean IS_NATIVE_IO_AVAIL;
+  static {
+    IS_NATIVE_IO_AVAIL = NativeIO.isAvailable();
+    if (Path.WINDOWS && !IS_NATIVE_IO_AVAIL) {
+      LOG.warn("Data node cannot fully support concurrent reading"
+          + " and writing without native code extensions on Windows.");
+    }
+  }
 
   /**
    * Constructor
@@ -190,14 +199,14 @@ abstract public class LocalReplica extends ReplicaInfo {
     File tmpFile = DatanodeUtil.createTmpFile(b, DatanodeUtil.getUnlinkTmpFile(file));
     try (FileInputStream in = new FileInputStream(file)) {
       try (FileOutputStream out = new FileOutputStream(tmpFile)){
-        copyBytes(in, out, 16 * 1024);
+        IOUtils.copyBytes(in, out, 16 * 1024);
       }
       if (file.length() != tmpFile.length()) {
         throw new IOException("Copy of file " + file + " size " + file.length()+
                               " into file " + tmpFile +
                               " resulted in a size of " + tmpFile.length());
       }
-      replaceFile(tmpFile, file);
+      FileUtil.replaceFile(tmpFile, file);
     } catch (IOException e) {
       boolean done = tmpFile.delete();
       if (!done) {
@@ -232,13 +241,13 @@ abstract public class LocalReplica extends ReplicaInfo {
     }
     File meta = getMetaFile();
 
-    int linkCount = getHardLinkCount(file);
+    int linkCount = HardLink.getLinkCount(file);
     if (linkCount > 1) {
       DataNode.LOG.info("Breaking hardlink for " + linkCount + "x-linked " +
           "block " + this);
       breakHardlinks(file, this);
     }
-    if (getHardLinkCount(meta) > 1) {
+    if (HardLink.getLinkCount(meta) > 1) {
       breakHardlinks(meta, this);
     }
     return true;
@@ -251,7 +260,18 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public InputStream getDataInputStream(long seekOffset) throws IOException {
-    return getDataInputStream(getBlockFile(), seekOffset);
+
+    File blockFile = getBlockFile();
+    if (IS_NATIVE_IO_AVAIL) {
+      return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
+    } else {
+      try {
+        return FsDatasetUtil.openAndSeek(blockFile, seekOffset);
+      } catch (FileNotFoundException fnfe) {
+        throw new IOException("Block " + this + " is not valid. " +
+            "Expected block file at " + blockFile + " does not exist.");
+      }
+    }
   }
 
   @Override
@@ -266,7 +286,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean deleteBlockData() {
-    return fullyDelete(getBlockFile());
+    return getBlockFile().delete();
   }
 
   @Override
@@ -300,7 +320,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean deleteMetadata() {
-    return fullyDelete(getMetaFile());
+    return getMetaFile().delete();
   }
 
   @Override
@@ -320,7 +340,7 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   private boolean renameFile(File srcfile, File destfile) throws IOException {
     try {
-      rename(srcfile, destfile);
+      NativeIO.renameTo(srcfile, destfile);
       return true;
     } catch (IOException e) {
       throw new IOException("Failed to move block file for " + this
@@ -347,14 +367,22 @@ abstract public class LocalReplica extends ReplicaInfo {
 
   @Override
   public boolean getPinning(LocalFileSystem localFS) throws IOException {
-    return getPinning(localFS, new Path(getBlockFile().getAbsolutePath()));
+    FileStatus fss =
+        localFS.getFileStatus(new Path(getBlockFile().getAbsolutePath()));
+    return fss.getPermission().getStickyBit();
   }
 
   @Override
   public void setPinning(LocalFileSystem localFS) throws IOException {
     File f = getBlockFile();
     Path p = new Path(f.getAbsolutePath());
-    setPinning(localFS, p);
+
+    FsPermission oldPermission = localFS.getFileStatus(
+        new Path(f.getAbsolutePath())).getPermission();
+    //sticky bit is used for pinning purpose
+    FsPermission permission = new FsPermission(oldPermission.getUserAction(),
+        oldPermission.getGroupAction(), oldPermission.getOtherAction(), true);
+    localFS.setPermission(p, permission);
   }
 
   @Override
@@ -370,7 +398,7 @@ abstract public class LocalReplica extends ReplicaInfo {
     }
     try {
       // calling renameMeta on the ReplicaInfo doesn't work here
-      rename(oldmeta, newmeta);
+      NativeIO.renameTo(oldmeta, newmeta);
     } catch (IOException e) {
       setGenerationStamp(oldGS); // restore old GS
       throw new IOException("Block " + this + " reopen failed. " +
@@ -389,113 +417,7 @@ abstract public class LocalReplica extends ReplicaInfo {
     return info.getBlockFile().compareTo(getBlockFile());
   }
 
-  @Override
-  public void copyMetadata(URI destination) throws IOException {
-    //for local replicas, we assume the destination URI is file
-    nativeCopyFileUnbuffered(getMetaFile(), new File(destination), true);
-  }
-
-  @Override
-  public void copyBlockdata(URI destination) throws IOException {
-    //for local replicas, we assume the destination URI is file
-    nativeCopyFileUnbuffered(getBlockFile(), new File(destination), true);
-  }
-
-  public void renameMeta(File newMetaFile) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + getMetaFile() + " to " + newMetaFile);
-    }
-    renameFile(getMetaFile(), newMetaFile);
-  }
-
-  public void renameBlock(File newBlockFile) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming " + getBlockFile() + " to " + newBlockFile
-          + ", file length=" + getBlockFile().length());
-    }
-    renameFile(getBlockFile(), newBlockFile);
-  }
-
-  public static void rename(File from, File to) throws IOException {
-    Storage.rename(from, to);
-  }
-
-  /**
-   * Get input stream for a local file and optionally seek to the offset.
-   * @param f path to the file
-   * @param seekOffset offset to seek
-   * @return
-   * @throws IOException
-   */
-  private FileInputStream getDataInputStream(File f, long seekOffset)
-      throws IOException {
-    FileInputStream fis;
-    if (NativeIO.isAvailable()) {
-      fis = NativeIO.getShareDeleteFileInputStream(f, seekOffset);
-    } else {
-      try {
-        fis = FsDatasetUtil.openAndSeek(f, seekOffset);
-      } catch (FileNotFoundException fnfe) {
-        throw new IOException("Expected block file at " + f +
-            " does not exist.");
-      }
-    }
-    return fis;
-  }
-
-  private void nativeCopyFileUnbuffered(File srcFile, File destFile,
-      boolean preserveFileDate) throws IOException {
-    Storage.nativeCopyFileUnbuffered(srcFile, destFile, preserveFileDate);
-  }
-
-  private void copyBytes(InputStream in, OutputStream out, int
-      buffSize) throws IOException{
-    IOUtils.copyBytes(in, out, buffSize);
-  }
-
-  private void replaceFile(File src, File target) throws IOException {
-    FileUtil.replaceFile(src, target);
-  }
-
-  public static boolean fullyDelete(final File dir) {
-    boolean result = DataStorage.fullyDelete(dir);
-    return result;
-  }
-
-  public static int getHardLinkCount(File fileName) throws IOException {
-    int linkCount = HardLink.getLinkCount(fileName);
-    return linkCount;
-  }
-
-  /**
-   *  Get pin status of a file by checking the sticky bit.
-   * @param localFS local file system
-   * @param path path to be checked
-   * @return
-   * @throws IOException
-   */
-  public boolean getPinning(LocalFileSystem localFS, Path path) throws
-      IOException {
-    boolean stickyBit =
-        localFS.getFileStatus(path).getPermission().getStickyBit();
-    return stickyBit;
-  }
-
-  /**
-   * Set sticky bit on path to pin file.
-   * @param localFS local file system
-   * @param path path to be pinned with sticky bit
-   * @throws IOException
-   */
-  public void setPinning(LocalFileSystem localFS, Path path) throws
-      IOException {
-    FsPermission oldPermission = localFS.getFileStatus(path).getPermission();
-    FsPermission permission = new FsPermission(oldPermission.getUserAction(),
-        oldPermission.getGroupAction(), oldPermission.getOtherAction(), true);
-    localFS.setPermission(path, permission);
-  }
-
-  public static void truncateBlock(File blockFile, File metaFile,
+  static public void truncateBlock(File blockFile, File metaFile,
       long oldlen, long newlen) throws IOException {
     LOG.info("truncateBlock: blockFile=" + blockFile
         + ", metaFile=" + metaFile
@@ -545,4 +467,19 @@ abstract public class LocalReplica extends ReplicaInfo {
       metaRAF.close();
     }
   }
+
+  @Override
+  public void copyMetadata(URI destination) throws IOException {
+    //for local replicas, we assume the destination URI is file
+    Storage.nativeCopyFileUnbuffered(getMetaFile(),
+        new File(destination), true);
+  }
+
+  @Override
+  public void copyBlockdata(URI destination) throws IOException {
+    //for local replicas, we assume the destination URI is file
+    Storage.nativeCopyFileUnbuffered(getBlockFile(),
+        new File(destination), true);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
index 1387155..bc7bc6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 
@@ -245,8 +246,7 @@ public class LocalReplicaInPipeline extends LocalReplica
 
   @Override // ReplicaInPipeline
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum, long slowLogThresholdMs)
-      throws IOException {
+      DataChecksum requestedChecksum) throws IOException {
     File blockFile = getBlockFile();
     File metaFile = getMetaFile();
     if (DataNode.LOG.isDebugEnabled()) {
@@ -313,7 +313,7 @@ public class LocalReplicaInPipeline extends LocalReplica
         crcOut.getChannel().position(crcDiskSize);
       }
       return new ReplicaOutputStreams(blockOut, crcOut, checksum,
-          getVolume().isTransientStorage(), slowLogThresholdMs);
+          getVolume().isTransientStorage());
     } catch (IOException e) {
       IOUtils.closeStream(blockOut);
       IOUtils.closeStream(metaRAF);
@@ -373,30 +373,40 @@ public class LocalReplicaInPipeline extends LocalReplica
           + " should be derived from LocalReplica");
     }
 
-    LocalReplica oldReplica = (LocalReplica) oldReplicaInfo;
-    File oldmeta = oldReplica.getMetaFile();
+    LocalReplica localReplica = (LocalReplica) oldReplicaInfo;
+
+    File oldmeta = localReplica.getMetaFile();
     File newmeta = getMetaFile();
 
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming " + oldmeta + " to " + newmeta);
+    }
     try {
-      oldReplica.renameMeta(newmeta);
+      NativeIO.renameTo(oldmeta, newmeta);
     } catch (IOException e) {
       throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
                             " Unable to move meta file  " + oldmeta +
                             " to rbw dir " + newmeta, e);
     }
 
+    File blkfile = localReplica.getBlockFile();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming " + blkfile + " to " + newBlkFile
+          + ", file length=" + blkfile.length());
+    }
     try {
-      oldReplica.renameBlock(newBlkFile);
+      NativeIO.renameTo(blkfile, newBlkFile);
     } catch (IOException e) {
       try {
-        renameMeta(oldmeta);
+        NativeIO.renameTo(newmeta, oldmeta);
       } catch (IOException ex) {
         LOG.warn("Cannot move meta file " + newmeta +
             "back to the finalized directory " + oldmeta, ex);
       }
       throw new IOException("Block " + oldReplicaInfo + " reopen failed. " +
-          " Unable to move block file " + oldReplica.getBlockFile() +
-          " to rbw dir " + newBlkFile, e);
+                              " Unable to move block file " + blkfile +
+                              " to rbw dir " + newBlkFile, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
index 5fdbec0..efa6ea6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
@@ -69,13 +69,11 @@ public interface ReplicaInPipeline extends Replica {
    *
    * @param isCreate if it is for creation
    * @param requestedChecksum the checksum the writer would prefer to use
-   * @param slowLogThresholdMs slow io threshold for logging
    * @return output streams for writing
    * @throws IOException if any error occurs
    */
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum, long slowLogThresholdMs)
-      throws IOException;
+      DataChecksum requestedChecksum) throws IOException;
 
   /**
    * Create an output stream to write restart metadata in case of datanode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 30f045f..57ec2b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.File;
+import java.io.FileDescriptor;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -604,7 +605,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * submit a sync_file_range request to AsyncDiskService.
    */
   void submitBackgroundSyncFileRangeRequest(final ExtendedBlock block,
-      final ReplicaOutputStreams outs, final long offset, final long nbytes,
+      final FileDescriptor fd, final long offset, final long nbytes,
       final int flags);
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
index 54d0e96..227179d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaInputStreams.java
@@ -18,45 +18,24 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 import java.io.Closeable;
-import java.io.FileDescriptor;
-import java.io.FileInputStream;
 import java.io.InputStream;
-import java.io.IOException;
 
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
-import org.apache.hadoop.io.nativeio.NativeIOException;
-import org.slf4j.Logger;
 
 /**
  * Contains the input streams for the data and checksum of a replica.
  */
 public class ReplicaInputStreams implements Closeable {
-  public static final Logger LOG = DataNode.LOG;
-
-  private InputStream dataIn;
-  private InputStream checksumIn;
-  private FsVolumeReference volumeRef;
-  private FileDescriptor dataInFd = null;
+  private final InputStream dataIn;
+  private final InputStream checksumIn;
+  private final FsVolumeReference volumeRef;
 
   /** Create an object with a data input stream and a checksum input stream. */
-  public ReplicaInputStreams(InputStream dataStream,
-      InputStream checksumStream, FsVolumeReference volumeRef) {
+  public ReplicaInputStreams(InputStream dataStream, InputStream checksumStream,
+      FsVolumeReference volumeRef) {
     this.volumeRef = volumeRef;
     this.dataIn = dataStream;
     this.checksumIn = checksumStream;
-    if (dataIn instanceof FileInputStream) {
-      try {
-        dataInFd = ((FileInputStream) dataIn).getFD();
-      } catch (Exception e) {
-        LOG.warn("Could not get file descriptor for inputstream of class " +
-            this.dataIn.getClass());
-      }
-    } else {
-      LOG.debug("Could not get file descriptor for inputstream of class " +
-          this.dataIn.getClass());
-    }
   }
 
   /** @return the data input stream. */
@@ -69,81 +48,10 @@ public class ReplicaInputStreams implements Closeable {
     return checksumIn;
   }
 
-  public FileDescriptor getDataInFd() {
-    return dataInFd;
-  }
-
-  public FsVolumeReference getVolumeRef() {
-    return volumeRef;
-  }
-
-  public void readDataFully(byte[] buf, int off, int len)
-      throws IOException {
-    IOUtils.readFully(dataIn, buf, off, len);
-  }
-
-  public void readChecksumFully(byte[] buf, int off, int len)
-      throws IOException {
-    IOUtils.readFully(checksumIn, buf, off, len);
-  }
-
-  public void skipDataFully(long len) throws IOException {
-    IOUtils.skipFully(dataIn, len);
-  }
-
-  public void skipChecksumFully(long len) throws IOException {
-    IOUtils.skipFully(checksumIn, len);
-  }
-
-  public void closeChecksumStream() throws IOException {
-    IOUtils.closeStream(checksumIn);
-    checksumIn = null;
-  }
-
-  public void dropCacheBehindReads(String identifier, long offset, long len,
-      int flags) throws NativeIOException {
-    assert this.dataInFd != null : "null dataInFd!";
-    NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-        identifier, dataInFd, offset, len, flags);
-  }
-
-  public void closeStreams() throws IOException {
-    IOException ioe = null;
-    if(checksumIn!=null) {
-      try {
-        checksumIn.close(); // close checksum file
-      } catch (IOException e) {
-        ioe = e;
-      }
-      checksumIn = null;
-    }
-    if(dataIn!=null) {
-      try {
-        dataIn.close(); // close data file
-      } catch (IOException e) {
-        ioe = e;
-      }
-      dataIn = null;
-      dataInFd = null;
-    }
-    if (volumeRef != null) {
-      IOUtils.cleanup(null, volumeRef);
-      volumeRef = null;
-    }
-    // throw IOException if there is any
-    if(ioe!= null) {
-      throw ioe;
-    }
-  }
-
   @Override
   public void close() {
     IOUtils.closeStream(dataIn);
-    dataIn = null;
-    dataInFd = null;
     IOUtils.closeStream(checksumIn);
-    checksumIn = null;
     IOUtils.cleanup(null, volumeRef);
-    volumeRef = null;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
index a66847a..bd1461a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java
@@ -18,62 +18,32 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 
 import java.io.Closeable;
-import java.io.FileDescriptor;
 import java.io.FileOutputStream;
 import java.io.OutputStream;
 import java.io.IOException;
 
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.nativeio.NativeIO;
-import org.apache.hadoop.io.nativeio.NativeIOException;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
 
 /**
  * Contains the output streams for the data and checksum of a replica.
  */
 public class ReplicaOutputStreams implements Closeable {
-  public static final Logger LOG = DataNode.LOG;
-
-  private FileDescriptor outFd = null;
-  /** Stream to block. */
-  private OutputStream dataOut;
-  /** Stream to checksum. */
+  private final OutputStream dataOut;
   private final OutputStream checksumOut;
   private final DataChecksum checksum;
   private final boolean isTransientStorage;
-  private final long slowLogThresholdMs;
 
   /**
    * Create an object with a data output stream, a checksum output stream
    * and a checksum.
    */
-  public ReplicaOutputStreams(OutputStream dataOut,
-      OutputStream checksumOut, DataChecksum checksum,
-      boolean isTransientStorage, long slowLogThresholdMs) {
+  public ReplicaOutputStreams(OutputStream dataOut, OutputStream checksumOut,
+      DataChecksum checksum, boolean isTransientStorage) {
     this.dataOut = dataOut;
+    this.checksumOut = checksumOut;
     this.checksum = checksum;
-    this.slowLogThresholdMs = slowLogThresholdMs;
     this.isTransientStorage = isTransientStorage;
-    this.checksumOut = checksumOut;
-
-    try {
-      if (this.dataOut instanceof FileOutputStream) {
-        this.outFd = ((FileOutputStream)this.dataOut).getFD();
-      } else {
-        LOG.debug("Could not get file descriptor for outputstream of class " +
-            this.dataOut.getClass());
-      }
-    } catch (IOException e) {
-      LOG.warn("Could not get file descriptor for outputstream of class " +
-          this.dataOut.getClass());
-    }
-  }
-
-  public FileDescriptor getOutFd() {
-    return outFd;
   }
 
   /** @return the data output stream. */
@@ -102,17 +72,12 @@ public class ReplicaOutputStreams implements Closeable {
     IOUtils.closeStream(checksumOut);
   }
 
-  public void closeDataStream() throws IOException {
-    dataOut.close();
-    dataOut = null;
-  }
-
   /**
    * Sync the data stream if it supports it.
    */
   public void syncDataOut() throws IOException {
     if (dataOut instanceof FileOutputStream) {
-      sync((FileOutputStream)dataOut);
+      ((FileOutputStream)dataOut).getChannel().force(true);
     }
   }
   
@@ -121,68 +86,8 @@ public class ReplicaOutputStreams implements Closeable {
    */
   public void syncChecksumOut() throws IOException {
     if (checksumOut instanceof FileOutputStream) {
-      sync((FileOutputStream)checksumOut);
+      ((FileOutputStream)checksumOut).getChannel().force(true);
     }
   }
 
-  /**
-   * Flush the data stream if it supports it.
-   */
-  public void flushDataOut() throws IOException {
-    flush(dataOut);
-  }
-
-  /**
-   * Flush the checksum stream if it supports it.
-   */
-  public void flushChecksumOut() throws IOException {
-    flush(checksumOut);
-  }
-
-  private void flush(OutputStream dos) throws IOException {
-    long begin = Time.monotonicNow();
-    dos.flush();
-    long duration = Time.monotonicNow() - begin;
-    LOG.trace("ReplicaOutputStreams#flush takes {} ms.", duration);
-    if (duration > slowLogThresholdMs) {
-      LOG.warn("Slow flush took {} ms (threshold={} ms)", duration,
-          slowLogThresholdMs);
-    }
-  }
-
-  private void sync(FileOutputStream fos) throws IOException {
-    long begin = Time.monotonicNow();
-    fos.getChannel().force(true);
-    long duration = Time.monotonicNow() - begin;
-    LOG.trace("ReplicaOutputStreams#sync takes {} ms.", duration);
-    if (duration > slowLogThresholdMs) {
-      LOG.warn("Slow fsync took {} ms (threshold={} ms)", duration,
-          slowLogThresholdMs);
-    }
-  }
-
-  public long writeToDisk(byte[] b, int off, int len) throws IOException {
-    long begin = Time.monotonicNow();
-    dataOut.write(b, off, len);
-    long duration = Time.monotonicNow() - begin;
-    LOG.trace("DatanodeIO#writeToDisk takes {} ms.", duration);
-    if (duration > slowLogThresholdMs) {
-      LOG.warn("Slow BlockReceiver write data to disk cost: {} ms " +
-          "(threshold={} ms)", duration, slowLogThresholdMs);
-    }
-    return duration;
-  }
-
-  public void syncFileRangeIfPossible(long offset, long nbytes,
-      int flags) throws NativeIOException {
-    assert this.outFd != null : "null outFd!";
-    NativeIO.POSIX.syncFileRangeIfPossible(outFd, offset, nbytes, flags);
-  }
-
-  public void dropCacheBehindWrites(String identifier,
-      long offset, long len, int flags) throws NativeIOException {
-    assert this.outFd != null : "null outFd!";
-    NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
-        identifier, outFd, offset, len, flags);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 8323140..29dbb29 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -49,13 +49,11 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
-import org.apache.hadoop.hdfs.server.datanode.LocalReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaTracker.RamDiskReplica;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
-
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker;
@@ -147,7 +145,7 @@ class BlockPoolSlice {
     //
     this.tmpDir = new File(bpDir, DataStorage.STORAGE_DIR_TMP);
     if (tmpDir.exists()) {
-      DataStorage.fullyDelete(tmpDir);
+      FileUtil.fullyDelete(tmpDir);
     }
     this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
     if (!rbwDir.mkdirs()) {  // create rbw directory if not exist
@@ -438,7 +436,7 @@ class BlockPoolSlice {
 
           final File targetMetaFile = new File(targetDir, metaFile.getName());
           try {
-            LocalReplica.rename(metaFile, targetMetaFile);
+            NativeIO.renameTo(metaFile, targetMetaFile);
           } catch (IOException e) {
             LOG.warn("Failed to move meta file from "
                 + metaFile + " to " + targetMetaFile, e);
@@ -448,7 +446,7 @@ class BlockPoolSlice {
 
           final File targetBlockFile = new File(targetDir, blockFile.getName());
           try {
-            LocalReplica.rename(blockFile, targetBlockFile);
+            NativeIO.renameTo(blockFile, targetBlockFile);
           } catch (IOException e) {
             LOG.warn("Failed to move block file from "
                 + blockFile + " to " + targetBlockFile, e);
@@ -690,7 +688,6 @@ class BlockPoolSlice {
    * @return the number of valid bytes
    */
   private long validateIntegrityAndSetLength(File blockFile, long genStamp) {
-    ReplicaInputStreams ris = null;
     DataInputStream checksumIn = null;
     InputStream blockIn = null;
     try {
@@ -717,22 +714,21 @@ class BlockPoolSlice {
       if (numChunks == 0) {
         return 0;
       }
+      IOUtils.skipFully(checksumIn, (numChunks-1)*checksumSize);
       blockIn = new FileInputStream(blockFile);
-      ris = new ReplicaInputStreams(blockIn, checksumIn,
-          volume.obtainReference());
-      ris.skipChecksumFully((numChunks-1)*checksumSize);
       long lastChunkStartPos = (numChunks-1)*bytesPerChecksum;
-      ris.skipDataFully(lastChunkStartPos);
+      IOUtils.skipFully(blockIn, lastChunkStartPos);
       int lastChunkSize = (int)Math.min(
           bytesPerChecksum, blockFileLen-lastChunkStartPos);
       byte[] buf = new byte[lastChunkSize+checksumSize];
-      ris.readChecksumFully(buf, lastChunkSize, checksumSize);
-      ris.readDataFully(buf, 0, lastChunkSize);
+      checksumIn.readFully(buf, lastChunkSize, checksumSize);
+      IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
+
       checksum.update(buf, 0, lastChunkSize);
       long validFileLength;
       if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
         validFileLength = lastChunkStartPos + lastChunkSize;
-      } else { // last chunk is corrupt
+      } else { // last chunck is corrupt
         validFileLength = lastChunkStartPos;
       }
 
@@ -752,12 +748,8 @@ class BlockPoolSlice {
       FsDatasetImpl.LOG.warn(e);
       return 0;
     } finally {
-      if (ris != null) {
-        ris.close();
-      } else {
-        IOUtils.closeStream(checksumIn);
-        IOUtils.closeStream(blockIn);
-      }
+      IOUtils.closeStream(checksumIn);
+      IOUtils.closeStream(blockIn);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
index 97dcf8d..b9c731b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import java.io.File;
+import java.io.FileDescriptor;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -37,9 +38,9 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIOException;
 
 /**
@@ -201,13 +202,13 @@ class FsDatasetAsyncDiskService {
   }
 
   public void submitSyncFileRangeRequest(FsVolumeImpl volume,
-      final ReplicaOutputStreams streams, final long offset, final long nbytes,
+      final FileDescriptor fd, final long offset, final long nbytes,
       final int flags) {
     execute(volume, new Runnable() {
       @Override
       public void run() {
         try {
-          streams.syncFileRangeIfPossible(offset, nbytes, flags);
+          NativeIO.POSIX.syncFileRangeIfPossible(fd, offset, nbytes, flags);
         } catch (NativeIOException e) {
           LOG.warn("sync_file_range error", e);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 6065df2..954d6ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -21,6 +21,7 @@ import java.io.BufferedOutputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.File;
+import java.io.FileDescriptor;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -2754,9 +2755,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
   @Override
   public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
-      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
+      FileDescriptor fd, long offset, long nbytes, int flags) {
     FsVolumeImpl fsVolumeImpl = this.getVolume(block);
-    asyncDiskService.submitSyncFileRangeRequest(fsVolumeImpl, outs, offset,
+    asyncDiskService.submitSyncFileRangeRequest(fsVolumeImpl, fd, offset,
         nbytes, flags);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 08564de..a231e03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -1067,7 +1067,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
         DataStorage.STORAGE_DIR_LAZY_PERSIST);
     File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
     if (force) {
-      DataStorage.fullyDelete(bpDir);
+      FileUtil.fullyDelete(bpDir);
     } else {
       if (!rbwDir.delete()) {
         throw new IOException("Failed to delete " + rbwDir);
@@ -1081,7 +1081,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
           !FileUtil.fullyDelete(lazypersistDir)))) {
         throw new IOException("Failed to delete " + lazypersistDir);
       }
-      DataStorage.fullyDelete(tmpDir);
+      FileUtil.fullyDelete(tmpDir);
       for (File f : FileUtil.listFiles(bpCurrentDir)) {
         if (!f.delete()) {
           throw new IOException("Failed to delete " + f);
@@ -1437,3 +1437,4 @@ public class FsVolumeImpl implements FsVolumeSpi {
         replicaState);
   }
 }
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
index e963d41..20cec6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
@@ -701,7 +701,7 @@ public class TestFileAppend{
       ReplicaBeingWritten rbw =
           (ReplicaBeingWritten)replicaHandler.getReplica();
       ReplicaOutputStreams
-          outputStreams = rbw.createStreams(false, DEFAULT_CHECKSUM, 300);
+          outputStreams = rbw.createStreams(false, DEFAULT_CHECKSUM);
       OutputStream dataOutput = outputStreams.getDataOut();
 
       byte[] appendBytes = new byte[1];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index f0266ef..5d63d07 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -17,7 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.*;
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.URI;
 import java.nio.channels.ClosedChannelException;
 import java.util.Collection;
@@ -257,15 +261,14 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
     @Override
     synchronized public ReplicaOutputStreams createStreams(boolean isCreate, 
-        DataChecksum requestedChecksum, long slowLogThresholdMs)
-        throws IOException {
+        DataChecksum requestedChecksum) throws IOException {
       if (finalized) {
         throw new IOException("Trying to write to a finalized replica "
             + theBlock);
       } else {
         SimulatedOutputStream crcStream = new SimulatedOutputStream();
         return new ReplicaOutputStreams(oStream, crcStream, requestedChecksum,
-            volume.isTransientStorage(), slowLogThresholdMs);
+            volume.isTransientStorage());
       }
     }
 
@@ -1361,7 +1364,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override
   public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
-      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
+      FileDescriptor fd, long offset, long nbytes, int flags) {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 8439991..619eda0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -673,7 +673,7 @@ public class TestBlockRecovery {
     ReplicaOutputStreams streams = null;
     try {
       streams = replicaInfo.createStreams(true,
-          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));
       streams.getChecksumOut().write('a');
       dn.data.initReplicaRecovery(new RecoveringBlock(block, null, RECOVERY_ID+1));
       BlockRecoveryWorker.RecoveryTaskContiguous RecoveryTaskContiguous =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
index fa980c2..4e724bc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
@@ -83,7 +83,7 @@ public class TestSimulatedFSDataset {
       ReplicaInPipeline bInfo = fsdataset.createRbw(
           StorageType.DEFAULT, b, false).getReplica();
       ReplicaOutputStreams out = bInfo.createStreams(true,
-          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512), 300);
+          DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 512));
       try {
         OutputStream dataOut  = out.getDataOut();
         assertEquals(0, fsdataset.getLength(b));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 2417c9d..4166346 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -318,8 +318,8 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block,
-      ReplicaOutputStreams outs, long offset, long nbytes, int flags) {
+  public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block, FileDescriptor fd, long offset, long nbytes, int flags) {
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcedb72a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
index 6fa2830..90c3b8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java
@@ -58,10 +58,8 @@ public class ExternalReplicaInPipeline implements ReplicaInPipeline {
 
   @Override
   public ReplicaOutputStreams createStreams(boolean isCreate,
-      DataChecksum requestedChecksum, long slowLogThresholdMs)
-      throws IOException {
-    return new ReplicaOutputStreams(null, null, requestedChecksum, false,
-        slowLogThresholdMs);
+      DataChecksum requestedChecksum) throws IOException {
+    return new ReplicaOutputStreams(null, null, requestedChecksum, false);
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[50/50] [abbrv] hadoop git commit: Merge branch 'trunk' into s3guard/HADOOP-13345

Posted by st...@apache.org.
Merge branch 'trunk' into s3guard/HADOOP-13345


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

Branch: refs/heads/HADOOP-13345
Commit: 881de1fba7299fb534d4cb3aae467ae2dd3dd9ee
Parents: 013a3c4 c265515
Author: Steve Loughran <st...@apache.org>
Authored: Thu Dec 8 17:58:10 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Dec 8 17:58:10 2016 +0000

----------------------------------------------------------------------
 BUILDING.txt                                    |    11 +-
 LICENSE.txt                                     |     2 +-
 dev-support/bin/qbt                             |     0
 .../util/ZKSignerSecretProvider.java            |     2 +-
 hadoop-common-project/hadoop-common/pom.xml     |     4 +
 .../apache/hadoop/crypto/key/KeyProvider.java   |    28 +
 .../crypto/key/KeyProviderCryptoExtension.java  |    94 +-
 .../crypto/key/kms/KMSClientProvider.java       |    77 +-
 .../hadoop/crypto/key/kms/KMSRESTConstants.java |     1 +
 .../key/kms/LoadBalancingKMSClientProvider.java |    18 +
 .../org/apache/hadoop/fs/FilterFileSystem.java  |     7 +
 .../org/apache/hadoop/fs/LocalDirAllocator.java |     2 +-
 .../apache/hadoop/fs/shell/CommandFormat.java   |     2 +-
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |    18 +
 .../io/erasurecode/ErasureCodeConstants.java    |     3 +
 .../io/erasurecode/rawcoder/util/GF256.java     |     2 +-
 .../org/apache/hadoop/io/file/tfile/TFile.java  |     2 +-
 .../org/apache/hadoop/io/file/tfile/Utils.java  |     2 +-
 .../org/apache/hadoop/net/NetworkTopology.java  |   162 +-
 .../java/org/apache/hadoop/security/KDiag.java  |     2 +-
 .../alias/LocalJavaKeyStoreProvider.java        |     2 +-
 .../security/ssl/FileBasedKeyStoresFactory.java |     6 +-
 .../security/ssl/ReloadingX509TrustManager.java |     2 +-
 .../src/main/native/gtest/gtest-all.cc          | 10403 ++++++++
 .../src/main/native/gtest/include/gtest/gtest.h | 21192 +++++++++++++++++
 .../resources/common-version-info.properties    |     4 +-
 .../conf/TestConfigurationFieldsBase.java       |    20 +-
 .../key/TestKeyProviderCryptoExtension.java     |   114 +
 .../fs/FileContextMainOperationsBaseTest.java   |    21 +-
 .../hadoop/fs/FileSystemContractBaseTest.java   |    17 +-
 .../apache/hadoop/fs/TestFilterFileSystem.java  |    12 +-
 .../fs/viewfs/ViewFileSystemBaseTest.java       |    29 +
 .../ssl/TestReloadingX509TrustManager.java      |    18 +
 ...rKeyGeneratorKeyProviderCryptoExtension.java |     7 +-
 .../hadoop/crypto/key/kms/server/KMS.java       |    38 +-
 .../hadoop/crypto/key/kms/server/KMSAudit.java  |     4 +-
 .../hadoop/crypto/key/kms/server/KMSWebApp.java |     7 +-
 .../kms/server/KeyAuthorizationKeyProvider.java |    13 +
 .../hadoop-kms/src/site/markdown/index.md.vm    |    39 +-
 .../hadoop/crypto/key/kms/server/TestKMS.java   |    90 +-
 .../crypto/key/kms/server/TestKMSAudit.java     |    11 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     |     1 +
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |    49 +
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    46 +
 .../hadoop/hdfs/web/resources/GetOpParam.java   |     3 +
 .../hadoop/hdfs/web/resources/PostOpParam.java  |     2 +
 .../hadoop/hdfs/web/resources/PutOpParam.java   |     1 +
 .../hdfs/web/resources/StoragePolicyParam.java  |    43 +
 .../hadoop/fs/http/client/HttpFSFileSystem.java |    92 +-
 .../hadoop/fs/http/server/FSOperations.java     |   130 +
 .../http/server/HttpFSParametersProvider.java   |    23 +
 .../hadoop/fs/http/server/HttpFSServer.java     |    35 +
 .../src/site/markdown/ServerSetup.md.vm         |     2 +-
 .../src/site/markdown/index.md                  |     6 +-
 .../fs/http/client/BaseTestHttpFSWith.java      |    52 +-
 .../main/native/fuse-dfs/fuse_dfs_wrapper.sh    |     2 +-
 .../dev-support/findbugsExcludeFile.xml         |    27 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |     2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |    76 +-
 .../server/blockmanagement/BlockManager.java    |    79 +-
 .../server/blockmanagement/DatanodeManager.java |    12 +-
 .../hdfs/server/datanode/BPOfferService.java    |    10 +-
 .../hdfs/server/datanode/BPServiceActor.java    |     4 +-
 .../hdfs/server/datanode/BlockReceiver.java     |    66 +-
 .../hdfs/server/datanode/BlockSender.java       |   105 +-
 .../hadoop/hdfs/server/datanode/DNConf.java     |     4 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |     2 +-
 .../hdfs/server/datanode/DataStorage.java       |     5 +
 .../hdfs/server/datanode/DirectoryScanner.java  |    12 +-
 .../hdfs/server/datanode/LocalReplica.java      |   179 +-
 .../server/datanode/LocalReplicaInPipeline.java |    30 +-
 .../hdfs/server/datanode/ReplicaInPipeline.java |     4 +-
 .../datanode/checker/DatasetVolumeChecker.java  |   442 +
 .../server/datanode/fsdataset/FsDatasetSpi.java |    10 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |    12 +-
 .../datanode/fsdataset/ReplicaInputStreams.java |   102 +-
 .../fsdataset/ReplicaOutputStreams.java         |   107 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |    97 +-
 .../impl/FsDatasetAsyncDiskService.java         |     7 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |     5 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |    20 +-
 .../namenode/ErasureCodingPolicyManager.java    |     5 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |    72 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |    15 +-
 .../hdfs/server/namenode/FSNamesystem.java      |    35 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |     2 +-
 .../server/namenode/ha/StandbyCheckpointer.java |     4 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |    39 +-
 .../hdfs/server/protocol/NamespaceInfo.java     |    26 +
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |    27 +
 .../hadoop-hdfs/src/main/proto/HdfsServer.proto |     2 +
 .../src/main/resources/hdfs-default.xml         |    12 +-
 .../src/main/webapps/hdfs/dfshealth.html        |    12 +-
 .../src/main/webapps/hdfs/dfshealth.js          |    12 +-
 .../src/main/webapps/hdfs/explorer.html         |    13 +-
 .../src/main/webapps/hdfs/explorer.js           |    37 +-
 .../src/site/markdown/HDFSErasureCoding.md      |     2 +-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |   261 +
 .../TestDFSRSDefault10x4StripedInputStream.java |    35 +
 ...TestDFSRSDefault10x4StripedOutputStream.java |    36 +
 ...fault10x4StripedOutputStreamWithFailure.java |    36 +
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |     2 +-
 .../server/datanode/SimulatedFSDataset.java     |    15 +-
 .../server/datanode/TestBPOfferService.java     |    31 +
 .../hdfs/server/datanode/TestBlockRecovery.java |     2 +-
 .../server/datanode/TestDirectoryScanner.java   |    18 +-
 .../server/datanode/TestSimulatedFSDataset.java |     2 +-
 .../checker/TestDatasetVolumeChecker.java       |   261 +
 .../TestDatasetVolumeCheckerFailures.java       |   193 +
 .../extdataset/ExternalDatasetImpl.java         |     4 +-
 .../extdataset/ExternalReplicaInPipeline.java   |     6 +-
 .../datanode/extdataset/ExternalVolumeImpl.java |     7 +
 .../namenode/TestAddStripedBlockInFBR.java      |    34 +-
 .../hdfs/server/namenode/TestFSNamesystem.java  |    21 +
 .../server/namenode/TestFSNamesystemMBean.java  |    24 +
 .../namenode/metrics/TestNameNodeMetrics.java   |     2 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |    68 +
 .../hadoop/hdfs/web/resources/TestParam.java    |     8 +
 .../apache/hadoop/net/TestNetworkTopology.java  |    29 +-
 .../hadoop/tools/TestHdfsConfigFields.java      |     6 -
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |    11 -
 .../mapreduce/v2/app/job/impl/TaskImpl.java     |     1 +
 .../mapreduce/v2/app/webapp/AppController.java  |    34 +
 .../mapreduce/v2/app/webapp/ConfBlock.java      |     2 +-
 .../mapreduce/v2/app/webapp/TaskPage.java       |     2 +-
 .../mapreduce/v2/app/webapp/TasksBlock.java     |     2 +-
 .../mapreduce/v2/app/webapp/dao/JobInfo.java    |     4 +-
 .../apache/hadoop/mapreduce/v2/app/MRApp.java   |    14 -
 .../hadoop/mapreduce/v2/app/TestKill.java       |     2 +-
 .../v2/app/webapp/TestAppController.java        |    14 +
 .../hadoop-mapreduce-client-core/pom.xml        |    16 +
 .../mapreduce/v2/hs/webapp/HsJobsBlock.java     |     2 +-
 .../hadoop/mapreduce/v2/hs/webapp/HsWebApp.java |     2 +
 .../mapreduce/v2/hs/webapp/dao/JobInfo.java     |    15 +-
 .../mapreduce/v2/hs/webapp/dao/TestJobInfo.java |    29 +-
 .../hadoop-mapreduce-client-nativetask/pom.xml  |     2 -
 .../src/CMakeLists.txt                          |     8 +-
 .../src/main/native/gtest/gtest-all.cc          | 10403 --------
 .../src/main/native/gtest/include/gtest/gtest.h | 21192 -----------------
 .../src/main/native/src/lib/Log.h               |     2 +-
 hadoop-maven-plugins/pom.xml                    |    12 +-
 .../maven/plugin/protoc/ProtocRunner.java       |     5 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |    23 +
 .../org/apache/hadoop/fs/s3a/Statistic.java     |     4 +-
 .../hadoop/fs/s3native/NativeS3FileSystem.java  |     7 +-
 .../fs/s3a/TestS3AExceptionTranslation.java     |    38 +
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |    24 +-
 .../org/apache/hadoop/fs/adl/TestAdlRead.java   |     6 +-
 .../apache/hadoop/fs/adl/TestListStatus.java    |     6 +-
 .../fs/adl/live/TestAdlContractAppendLive.java  |    11 +-
 .../fs/adl/live/TestAdlContractConcatLive.java  |    23 +-
 .../fs/adl/live/TestAdlContractCreateLive.java  |    19 +-
 .../fs/adl/live/TestAdlContractDeleteLive.java  |    11 +-
 .../live/TestAdlContractGetFileStatusLive.java  |    36 +
 .../fs/adl/live/TestAdlContractMkdirLive.java   |    25 +-
 .../fs/adl/live/TestAdlContractOpenLive.java    |    11 +-
 .../fs/adl/live/TestAdlContractRenameLive.java  |    30 +-
 .../fs/adl/live/TestAdlContractRootDirLive.java |    19 +-
 .../fs/adl/live/TestAdlContractSeekLive.java    |    11 +-
 .../live/TestAdlDifferentSizeWritesLive.java    |    69 +-
 .../live/TestAdlFileContextCreateMkdirLive.java |    67 +
 .../TestAdlFileContextMainOperationsLive.java   |    99 +
 .../adl/live/TestAdlFileSystemContractLive.java |    57 +-
 .../live/TestAdlInternalCreateNonRecursive.java |   134 +
 .../fs/adl/live/TestAdlPermissionLive.java      |   116 +
 .../adl/live/TestAdlSupportedCharsetInPath.java |   334 +
 .../apache/hadoop/fs/adl/live/TestMetadata.java |   111 +
 .../fs/azure/AzureNativeFileSystemStore.java    |    31 +-
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |    25 +-
 .../hadoop/fs/azure/NativeFileSystemStore.java  |    23 +-
 ...estNativeAzureFileSystemConcurrencyLive.java |   119 +
 .../GetClusterNodeLabelsResponse.java           |    50 +-
 .../yarn/api/records/ApplicationReport.java     |     6 +-
 .../yarn/api/records/ContainerStatus.java       |    40 +-
 .../hadoop/yarn/api/records/NodeReport.java     |    12 +-
 .../src/main/proto/yarn_protos.proto            |     7 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |    24 -
 .../yarn/client/api/impl/YarnClientImpl.java    |     2 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |     3 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |     2 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |     2 +-
 .../hadoop/yarn/client/TestGetGroups.java       |    36 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |     7 +-
 .../pb/GetClusterNodeLabelsResponsePBImpl.java  |    41 +-
 .../impl/pb/ApplicationReportPBImpl.java        |   107 +-
 .../api/impl/FileSystemTimelineWriter.java      |     5 +-
 .../client/api/impl/TimelineClientImpl.java     |     2 +-
 .../yarn/security/ContainerTokenIdentifier.java |    25 +
 .../security/YarnAuthorizationProvider.java     |    15 +
 .../state/InvalidStateTransitionException.java  |    22 +-
 .../state/InvalidStateTransitonException.java   |    19 +-
 .../org/apache/hadoop/yarn/util/FSDownload.java |     2 +-
 .../java/org/apache/hadoop/yarn/util/Times.java |     3 +
 .../org/apache/hadoop/yarn/webapp/Router.java   |    23 +-
 .../org/apache/hadoop/yarn/webapp/WebApp.java   |    13 +
 .../main/resources/yarn-version-info.properties |     2 +-
 .../client/impl/zk/RegistrySecurity.java        |     2 +-
 ...pplicationHistoryManagerOnTimelineStore.java |     2 +-
 .../WindowsSecureContainerExecutor.java         |     4 +-
 .../containermanager/AuxServices.java           |     4 +-
 .../containermanager/ContainerManagerImpl.java  |     2 +-
 .../localizer/ResourceLocalizationService.java  |     2 +-
 .../logaggregation/AppLogAggregatorImpl.java    |     2 +-
 .../server/resourcemanager/ClientRMService.java |    18 +-
 .../server/resourcemanager/RMAppManager.java    |   110 +-
 .../resourcemanager/RMAppManagerEvent.java      |    11 +
 .../resourcemanager/RMAppManagerEventType.java  |     3 +-
 .../server/resourcemanager/RMContextImpl.java   |    10 +-
 .../server/resourcemanager/RMServerUtils.java   |     3 +-
 .../resourcemanager/rmapp/RMAppEventType.java   |     1 -
 .../server/resourcemanager/rmapp/RMAppImpl.java |    61 +-
 .../resourcemanager/rmapp/RMAppMoveEvent.java   |    44 -
 .../scheduler/AbstractYarnScheduler.java        |    18 +-
 .../scheduler/SchedulerQueueManager.java        |    75 +
 .../scheduler/YarnScheduler.java                |    11 +
 .../scheduler/capacity/AbstractCSQueue.java     |    35 +-
 .../scheduler/capacity/CSQueue.java             |    10 +
 .../scheduler/capacity/CapacityScheduler.java   |   321 +-
 .../CapacitySchedulerConfiguration.java         |    22 +-
 .../capacity/CapacitySchedulerQueueManager.java |   361 +
 .../scheduler/capacity/LeafQueue.java           |    19 +-
 .../scheduler/capacity/ParentQueue.java         |    29 +-
 .../scheduler/fair/AllocationConfiguration.java |    38 +-
 .../fair/AllocationFileLoaderService.java       |    68 +-
 .../resourcemanager/scheduler/fair/FSQueue.java |    22 +-
 .../scheduler/fair/FSQueueMetrics.java          |    32 +-
 .../scheduler/fair/FairScheduler.java           |   113 +-
 .../resourcemanager/webapp/RMWebServices.java   |    10 +-
 .../resourcemanager/TestClientRMService.java    |     4 +-
 .../resourcemanager/TestMoveApplication.java    |    12 +-
 ...tyPreemptionPolicyForReservedContainers.java |   430 +
 .../rmapp/TestApplicationLifetimeMonitor.java   |    15 +-
 .../capacity/TestApplicationLimits.java         |    35 +-
 .../TestApplicationLimitsByPartition.java       |     7 +-
 .../TestCapacitySchedulerNodeLabelUpdate.java   |     1 +
 .../scheduler/capacity/TestChildQueueOrder.java |     9 +-
 .../scheduler/capacity/TestLeafQueue.java       |     9 +-
 .../scheduler/capacity/TestParentQueue.java     |    39 +-
 .../scheduler/capacity/TestQueueState.java      |    96 +
 .../scheduler/capacity/TestReservations.java    |     8 +-
 .../scheduler/capacity/TestUtils.java           |     2 +-
 .../scheduler/fair/TestFSQueueMetrics.java      |    69 +
 .../scheduler/fair/TestFairScheduler.java       |    93 +-
 .../TestRMWebServicesAppsModification.java      |    39 +-
 .../yarn/server/webproxy/ProxyUriUtils.java     |     2 +-
 .../yarn/server/webproxy/WebAppProxy.java       |     2 +-
 .../src/site/markdown/FairScheduler.md          |     8 +-
 247 files changed, 38744 insertions(+), 33142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/881de1fb/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/50] [abbrv] hadoop git commit: HDFS-11178. TestAddStripedBlockInFBR#testAddBlockInFullBlockReport fails frequently in trunk. Contributed By Yiqun Lin.

Posted by st...@apache.org.
HDFS-11178. TestAddStripedBlockInFBR#testAddBlockInFullBlockReport fails frequently in trunk. Contributed By Yiqun Lin.


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

Branch: refs/heads/HADOOP-13345
Commit: ed89856755fd20e814d3ba58e2c183a85a9389d3
Parents: 4dd4f3a
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Tue Dec 6 22:30:43 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Tue Dec 6 22:30:43 2016 +0530

----------------------------------------------------------------------
 .../namenode/TestAddStripedBlockInFBR.java      | 34 ++++++++++++++------
 1 file changed, 24 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed898567/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlockInFBR.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlockInFBR.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlockInFBR.java
index ab24a25..6e02372 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlockInFBR.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlockInFBR.java
@@ -30,9 +30,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -40,6 +39,8 @@ import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import com.google.common.base.Supplier;
+
 import java.io.IOException;
 
 public class TestAddStripedBlockInFBR {
@@ -98,14 +99,27 @@ public class TestAddStripedBlockInFBR {
     DFSTestUtil.createFile(dfs, ecFile,
         cellSize * dataBlocks, (short) 1, 0L);
 
-    // trigger dn's FBR. The FBR will add block-dn mapping.
-    DataNodeTestUtils.triggerBlockReport(dn);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+
+      @Override
+      public Boolean get() {
+        try {
+          // trigger dn's FBR. The FBR will add block-dn mapping.
+          cluster.triggerBlockReports();
+
+          // make sure NN has correct block-dn mapping
+          BlockInfoStriped blockInfo = (BlockInfoStriped) cluster
+              .getNamesystem().getFSDirectory().getINode(ecFile.toString())
+              .asFile().getLastBlock();
+          NumberReplicas nr = spy.countNodes(blockInfo);
+
+          return nr.excessReplicas() == 0 && nr.liveReplicas() == groupSize;
+        } catch (Exception ignored) {
+          // Ignore the exception
+        }
 
-    // make sure NN has correct block-dn mapping
-    BlockInfoStriped blockInfo = (BlockInfoStriped) cluster.getNamesystem()
-        .getFSDirectory().getINode(ecFile.toString()).asFile().getLastBlock();
-    NumberReplicas nr = spy.countNodes(blockInfo);
-    Assert.assertEquals(groupSize, nr.liveReplicas());
-    Assert.assertEquals(0, nr.excessReplicas());
+        return false;
+      }
+    }, 3000, 60000);
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/50] [abbrv] hadoop git commit: HADOOP-13861. Spelling errors in logging and exceptions for code. Contributed by Grant Sohn.

Posted by st...@apache.org.
HADOOP-13861. Spelling errors in logging and exceptions for code. Contributed by Grant Sohn.


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

Branch: refs/heads/HADOOP-13345
Commit: 7b988e88992528a0cac2ca8893652c5d4a90c6b9
Parents: 08a7253
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Dec 5 23:18:18 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Dec 5 23:18:18 2016 -0800

----------------------------------------------------------------------
 .../security/authentication/util/ZKSignerSecretProvider.java     | 2 +-
 .../src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java    | 2 +-
 .../src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java  | 2 +-
 .../org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java    | 2 +-
 .../src/main/java/org/apache/hadoop/io/file/tfile/TFile.java     | 2 +-
 .../src/main/java/org/apache/hadoop/io/file/tfile/Utils.java     | 2 +-
 .../src/main/java/org/apache/hadoop/net/NetworkTopology.java     | 4 ++--
 .../src/main/java/org/apache/hadoop/security/KDiag.java          | 2 +-
 .../apache/hadoop/security/alias/LocalJavaKeyStoreProvider.java  | 2 +-
 9 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
index 1d16b2d..48dfaaa 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
@@ -258,7 +258,7 @@ public class ZKSignerSecretProvider extends RolloverSignerSecretProvider {
     } catch (KeeperException.BadVersionException bve) {
       LOG.debug("Unable to push to znode; another server already did it");
     } catch (Exception ex) {
-      LOG.error("An unexpected exception occured pushing data to ZooKeeper",
+      LOG.error("An unexpected exception occurred pushing data to ZooKeeper",
               ex);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
index b14e1f0..1ed01ea 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
@@ -525,7 +525,7 @@ public class LocalDirAllocator {
         try {
           advance();
         } catch (IOException ie) {
-          throw new RuntimeException("Can't check existance of " + next, ie);
+          throw new RuntimeException("Can't check existence of " + next, ie);
         }
         if (result == null) {
           throw new NoSuchElementException();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
index 0aa3d65..bf30b22 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
@@ -248,7 +248,7 @@ public class CommandFormat {
     private static final long serialVersionUID = 0L;
 
     public DuplicatedOptionException(String duplicatedOption) {
-      super("option " + duplicatedOption + " already exsits!");
+      super("option " + duplicatedOption + " already exists!");
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java
index 03e21b6..35534f3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java
@@ -221,7 +221,7 @@ public final class GF256 {
         }
         if (j == n) {
           // Couldn't find means it's singular
-          throw new RuntimeException("Not invertble");
+          throw new RuntimeException("Not invertible");
         }
 
         for (int k = 0; k < n; k++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java
index f17be1a..56739c6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java
@@ -1789,7 +1789,7 @@ public class TFile {
         public int getKey(byte[] buf, int offset) throws IOException {
           if ((offset | (buf.length - offset - klen)) < 0) {
             throw new IndexOutOfBoundsException(
-                "Bufer not enough to store the key");
+                "Buffer not enough to store the key");
           }
           System.arraycopy(keyBuffer, 0, buf, offset, klen);
           return klen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java
index 12148ef..5743c66 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java
@@ -153,7 +153,7 @@ public final class Utils {
         out.writeLong(n);
         return;
       default:
-        throw new RuntimeException("Internel error");
+        throw new RuntimeException("Internal error");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
index 0e6c253..14c870d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
@@ -169,7 +169,7 @@ public class NetworkTopology {
       if (!isAncestor(n)) {
         throw new IllegalArgumentException(n.getName()
             + ", which is located at " + n.getNetworkLocation()
-            + ", is not a descendent of " + getPath(this));
+            + ", is not a descendant of " + getPath(this));
       }
       if (isParent(n)) {
         // this node is the parent of n; add n directly
@@ -231,7 +231,7 @@ public class NetworkTopology {
       if (!isAncestor(n)) {
         throw new IllegalArgumentException(n.getName()
             + ", which is located at " + n.getNetworkLocation()
-            + ", is not a descendent of " + getPath(this));
+            + ", is not a descendant of " + getPath(this));
       }
       if (isParent(n)) {
         // this node is the parent of n; remove n directly

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
index 542a502..c8d0b33 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
@@ -593,7 +593,7 @@ public class KDiag extends Configured implements Tool, Closeable {
 
     Keytab loadKeytab = Keytab.loadKeytab(kt);
     List<PrincipalName> principals = loadKeytab.getPrincipals();
-    println("keytab princial count: %d", principals.size());
+    println("keytab principal count: %d", principals.size());
     int entrySize = 0;
     for (PrincipalName princ : principals) {
       List<KeytabEntry> entries = loadKeytab.getKeytabEntries(princ);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b988e88/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/LocalJavaKeyStoreProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/LocalJavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/LocalJavaKeyStoreProvider.java
index 1891cd3..9ea9a57 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/LocalJavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/LocalJavaKeyStoreProvider.java
@@ -151,7 +151,7 @@ public final class LocalJavaKeyStoreProvider extends
   public void flush() throws IOException {
     super.flush();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Reseting permissions to '" + permissions + "'");
+      LOG.debug("Resetting permissions to '" + permissions + "'");
     }
     if (!Shell.WINDOWS) {
       Files.setPosixFilePermissions(Paths.get(file.getCanonicalPath()),


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[45/50] [abbrv] hadoop git commit: HDFS-11140. Directory Scanner should log startup message time correctly. Contributed by Yiqun Lin.

Posted by st...@apache.org.
HDFS-11140. Directory Scanner should log startup message time correctly. Contributed by Yiqun Lin.


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

Branch: refs/heads/HADOOP-13345
Commit: 9ef89ede2f18c76c601fd585cb9d47511f5fc3ed
Parents: deb4743
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 8 13:38:02 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 8 13:38:02 2016 +0900

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/DirectoryScanner.java   | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ef89ede/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index e2baf32..18188dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -37,6 +37,8 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -61,7 +63,7 @@ public class DirectoryScanner implements Runnable {
   private static final int MILLIS_PER_SECOND = 1000;
   private static final String START_MESSAGE =
       "Periodic Directory Tree Verification scan"
-      + " starting at %dms with interval of %dms";
+      + " starting at %s with interval of %dms";
   private static final String START_MESSAGE_WITH_THROTTLE = START_MESSAGE
       + " and throttle limit of %dms/s";
 
@@ -268,10 +270,12 @@ public class DirectoryScanner implements Runnable {
     String logMsg;
 
     if (throttleLimitMsPerSec < MILLIS_PER_SECOND) {
-      logMsg = String.format(START_MESSAGE_WITH_THROTTLE, firstScanTime,
-          scanPeriodMsecs, throttleLimitMsPerSec);
+      logMsg = String.format(START_MESSAGE_WITH_THROTTLE,
+          FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs,
+          throttleLimitMsPerSec);
     } else {
-      logMsg = String.format(START_MESSAGE, firstScanTime, scanPeriodMsecs);
+      logMsg = String.format(START_MESSAGE,
+          FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs);
     }
 
     LOG.info(logMsg);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/50] [abbrv] hadoop git commit: HADOOP-13835. Move Google Test Framework code from mapreduce to hadoop-common. Contributed by Varun Vasudev.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/include/gtest/gtest.h
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/include/gtest/gtest.h b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/include/gtest/gtest.h
deleted file mode 100644
index c04205d..0000000
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest/include/gtest/gtest.h
+++ /dev/null
@@ -1,21192 +0,0 @@
-// Copyright 2005, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Author: wan@google.com (Zhanyong Wan)
-//
-// The Google C++ Testing Framework (Google Test)
-//
-// This header file defines the public API for Google Test.  It should be
-// included by any test program that uses Google Test.
-//
-// IMPORTANT NOTE: Due to limitation of the C++ language, we have to
-// leave some internal implementation details in this header file.
-// They are clearly marked by comments like this:
-//
-//   // INTERNAL IMPLEMENTATION - DO NOT USE IN A USER PROGRAM.
-//
-// Such code is NOT meant to be used by a user directly, and is subject
-// to CHANGE WITHOUT NOTICE.  Therefore DO NOT DEPEND ON IT in a user
-// program!
-//
-// Acknowledgment: Google Test borrowed the idea of automatic test
-// registration from Barthelemy Dagenais' (barthelemy@prologique.com)
-// easyUnit framework.
-
-#ifndef GTEST_INCLUDE_GTEST_GTEST_H_
-#define GTEST_INCLUDE_GTEST_GTEST_H_
-
-#include <limits>
-#include <ostream>
-#include <vector>
-
-// Copyright 2005, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Authors: wan@google.com (Zhanyong Wan), eefacm@gmail.com (Sean Mcafee)
-//
-// The Google C++ Testing Framework (Google Test)
-//
-// This header file declares functions and macros used internally by
-// Google Test.  They are subject to change without notice.
-
-#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_INTERNAL_H_
-#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_INTERNAL_H_
-
-// Copyright 2005, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Authors: wan@google.com (Zhanyong Wan)
-//
-// Low-level types and utilities for porting Google Test to various
-// platforms.  All macros ending with _ and symbols defined in an
-// internal namespace are subject to change without notice.  Code
-// outside Google Test MUST NOT USE THEM DIRECTLY.  Macros that don't
-// end with _ are part of Google Test's public API and can be used by
-// code outside Google Test.
-//
-// This file is fundamental to Google Test.  All other Google Test source
-// files are expected to #include this.  Therefore, it cannot #include
-// any other Google Test header.
-
-#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_H_
-#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_H_
-
-// Environment-describing macros
-// -----------------------------
-//
-// Google Test can be used in many different environments.  Macros in
-// this section tell Google Test what kind of environment it is being
-// used in, such that Google Test can provide environment-specific
-// features and implementations.
-//
-// Google Test tries to automatically detect the properties of its
-// environment, so users usually don't need to worry about these
-// macros.  However, the automatic detection is not perfect.
-// Sometimes it's necessary for a user to define some of the following
-// macros in the build script to override Google Test's decisions.
-//
-// If the user doesn't define a macro in the list, Google Test will
-// provide a default definition.  After this header is #included, all
-// macros in this list will be defined to either 1 or 0.
-//
-// Notes to maintainers:
-//   - Each macro here is a user-tweakable knob; do not grow the list
-//     lightly.
-//   - Use #if to key off these macros.  Don't use #ifdef or "#if
-//     defined(...)", which will not work as these macros are ALWAYS
-//     defined.
-//
-//   GTEST_HAS_CLONE          - Define it to 1/0 to indicate that clone(2)
-//                              is/isn't available.
-//   GTEST_HAS_EXCEPTIONS     - Define it to 1/0 to indicate that exceptions
-//                              are enabled.
-//   GTEST_HAS_GLOBAL_STRING  - Define it to 1/0 to indicate that ::string
-//                              is/isn't available (some systems define
-//                              ::string, which is different to std::string).
-//   GTEST_HAS_GLOBAL_WSTRING - Define it to 1/0 to indicate that ::string
-//                              is/isn't available (some systems define
-//                              ::wstring, which is different to std::wstring).
-//   GTEST_HAS_POSIX_RE       - Define it to 1/0 to indicate that POSIX regular
-//                              expressions are/aren't available.
-//   GTEST_HAS_PTHREAD        - Define it to 1/0 to indicate that <pthread.h>
-//                              is/isn't available.
-//   GTEST_HAS_RTTI           - Define it to 1/0 to indicate that RTTI is/isn't
-//                              enabled.
-//   GTEST_HAS_STD_WSTRING    - Define it to 1/0 to indicate that
-//                              std::wstring does/doesn't work (Google Test can
-//                              be used where std::wstring is unavailable).
-//   GTEST_HAS_TR1_TUPLE      - Define it to 1/0 to indicate tr1::tuple
-//                              is/isn't available.
-//   GTEST_HAS_SEH            - Define it to 1/0 to indicate whether the
-//                              compiler supports Microsoft's "Structured
-//                              Exception Handling".
-//   GTEST_HAS_STREAM_REDIRECTION
-//                            - Define it to 1/0 to indicate whether the
-//                              platform supports I/O stream redirection using
-//                              dup() and dup2().
-//   GTEST_USE_OWN_TR1_TUPLE  - Define it to 1/0 to indicate whether Google
-//                              Test's own tr1 tuple implementation should be
-//                              used.  Unused when the user sets
-//                              GTEST_HAS_TR1_TUPLE to 0.
-//   GTEST_LANG_CXX11         - Define it to 1/0 to indicate that Google Test
-//                              is building in C++11/C++98 mode.
-//   GTEST_LINKED_AS_SHARED_LIBRARY
-//                            - Define to 1 when compiling tests that use
-//                              Google Test as a shared library (known as
-//                              DLL on Windows).
-//   GTEST_CREATE_SHARED_LIBRARY
-//                            - Define to 1 when compiling Google Test itself
-//                              as a shared library.
-
-// Platform-indicating macros
-// --------------------------
-//
-// Macros indicating the platform on which Google Test is being used
-// (a macro is defined to 1 if compiled on the given platform;
-// otherwise UNDEFINED -- it's never defined to 0.).  Google Test
-// defines these macros automatically.  Code outside Google Test MUST
-// NOT define them.
-//
-//   GTEST_OS_AIX      - IBM AIX
-//   GTEST_OS_CYGWIN   - Cygwin
-//   GTEST_OS_FREEBSD  - FreeBSD
-//   GTEST_OS_HPUX     - HP-UX
-//   GTEST_OS_LINUX    - Linux
-//     GTEST_OS_LINUX_ANDROID - Google Android
-//   GTEST_OS_MAC      - Mac OS X
-//     GTEST_OS_IOS    - iOS
-//   GTEST_OS_NACL     - Google Native Client (NaCl)
-//   GTEST_OS_OPENBSD  - OpenBSD
-//   GTEST_OS_QNX      - QNX
-//   GTEST_OS_SOLARIS  - Sun Solaris
-//   GTEST_OS_SYMBIAN  - Symbian
-//   GTEST_OS_WINDOWS  - Windows (Desktop, MinGW, or Mobile)
-//     GTEST_OS_WINDOWS_DESKTOP  - Windows Desktop
-//     GTEST_OS_WINDOWS_MINGW    - MinGW
-//     GTEST_OS_WINDOWS_MOBILE   - Windows Mobile
-//     GTEST_OS_WINDOWS_PHONE    - Windows Phone
-//     GTEST_OS_WINDOWS_RT       - Windows Store App/WinRT
-//   GTEST_OS_ZOS      - z/OS
-//
-// Among the platforms, Cygwin, Linux, Max OS X, and Windows have the
-// most stable support.  Since core members of the Google Test project
-// don't have access to other platforms, support for them may be less
-// stable.  If you notice any problems on your platform, please notify
-// googletestframework@googlegroups.com (patches for fixing them are
-// even more welcome!).
-//
-// It is possible that none of the GTEST_OS_* macros are defined.
-
-// Feature-indicating macros
-// -------------------------
-//
-// Macros indicating which Google Test features are available (a macro
-// is defined to 1 if the corresponding feature is supported;
-// otherwise UNDEFINED -- it's never defined to 0.).  Google Test
-// defines these macros automatically.  Code outside Google Test MUST
-// NOT define them.
-//
-// These macros are public so that portable tests can be written.
-// Such tests typically surround code using a feature with an #if
-// which controls that code.  For example:
-//
-// #if GTEST_HAS_DEATH_TEST
-//   EXPECT_DEATH(DoSomethingDeadly());
-// #endif
-//
-//   GTEST_HAS_COMBINE      - the Combine() function (for value-parameterized
-//                            tests)
-//   GTEST_HAS_DEATH_TEST   - death tests
-//   GTEST_HAS_PARAM_TEST   - value-parameterized tests
-//   GTEST_HAS_TYPED_TEST   - typed tests
-//   GTEST_HAS_TYPED_TEST_P - type-parameterized tests
-//   GTEST_IS_THREADSAFE    - Google Test is thread-safe.
-//   GTEST_USES_POSIX_RE    - enhanced POSIX regex is used. Do not confuse with
-//                            GTEST_HAS_POSIX_RE (see above) which users can
-//                            define themselves.
-//   GTEST_USES_SIMPLE_RE   - our own simple regex is used;
-//                            the above two are mutually exclusive.
-//   GTEST_CAN_COMPARE_NULL - accepts untyped NULL in EXPECT_EQ().
-
-// Misc public macros
-// ------------------
-//
-//   GTEST_FLAG(flag_name)  - references the variable corresponding to
-//                            the given Google Test flag.
-
-// Internal utilities
-// ------------------
-//
-// The following macros and utilities are for Google Test's INTERNAL
-// use only.  Code outside Google Test MUST NOT USE THEM DIRECTLY.
-//
-// Macros for basic C++ coding:
-//   GTEST_AMBIGUOUS_ELSE_BLOCKER_ - for disabling a gcc warning.
-//   GTEST_ATTRIBUTE_UNUSED_  - declares that a class' instances or a
-//                              variable don't have to be used.
-//   GTEST_DISALLOW_ASSIGN_   - disables operator=.
-//   GTEST_DISALLOW_COPY_AND_ASSIGN_ - disables copy ctor and operator=.
-//   GTEST_MUST_USE_RESULT_   - declares that a function's result must be used.
-//   GTEST_INTENTIONAL_CONST_COND_PUSH_ - start code section where MSVC C4127 is
-//                                        suppressed (constant conditional).
-//   GTEST_INTENTIONAL_CONST_COND_POP_  - finish code section where MSVC C4127
-//                                        is suppressed.
-//
-// C++11 feature wrappers:
-//
-//   testing::internal::move  - portability wrapper for std::move.
-//
-// Synchronization:
-//   Mutex, MutexLock, ThreadLocal, GetThreadCount()
-//                            - synchronization primitives.
-//
-// Template meta programming:
-//   is_pointer     - as in TR1; needed on Symbian and IBM XL C/C++ only.
-//   IteratorTraits - partial implementation of std::iterator_traits, which
-//                    is not available in libCstd when compiled with Sun C++.
-//
-// Smart pointers:
-//   scoped_ptr     - as in TR2.
-//
-// Regular expressions:
-//   RE             - a simple regular expression class using the POSIX
-//                    Extended Regular Expression syntax on UNIX-like
-//                    platforms, or a reduced regular exception syntax on
-//                    other platforms, including Windows.
-//
-// Logging:
-//   GTEST_LOG_()   - logs messages at the specified severity level.
-//   LogToStderr()  - directs all log messages to stderr.
-//   FlushInfoLog() - flushes informational log messages.
-//
-// Stdout and stderr capturing:
-//   CaptureStdout()     - starts capturing stdout.
-//   GetCapturedStdout() - stops capturing stdout and returns the captured
-//                         string.
-//   CaptureStderr()     - starts capturing stderr.
-//   GetCapturedStderr() - stops capturing stderr and returns the captured
-//                         string.
-//
-// Integer types:
-//   TypeWithSize   - maps an integer to a int type.
-//   Int32, UInt32, Int64, UInt64, TimeInMillis
-//                  - integers of known sizes.
-//   BiggestInt     - the biggest signed integer type.
-//
-// Command-line utilities:
-//   GTEST_DECLARE_*()  - declares a flag.
-//   GTEST_DEFINE_*()   - defines a flag.
-//   GetInjectableArgvs() - returns the command line as a vector of strings.
-//
-// Environment variable utilities:
-//   GetEnv()             - gets the value of an environment variable.
-//   BoolFromGTestEnv()   - parses a bool environment variable.
-//   Int32FromGTestEnv()  - parses an Int32 environment variable.
-//   StringFromGTestEnv() - parses a string environment variable.
-
-#include <ctype.h>   // for isspace, etc
-#include <stddef.h>  // for ptrdiff_t
-#include <stdlib.h>
-#include <stdio.h>
-#include <string.h>
-#ifndef _WIN32_WCE
-# include <sys/types.h>
-# include <sys/stat.h>
-#endif  // !_WIN32_WCE
-
-#if defined __APPLE__
-# include <AvailabilityMacros.h>
-# include <TargetConditionals.h>
-#endif
-
-#include <algorithm>  // NOLINT
-#include <iostream>  // NOLINT
-#include <sstream>  // NOLINT
-#include <string>  // NOLINT
-#include <utility>
-#include <vector>  // NOLINT
-
-// Copyright 2015, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// The Google C++ Testing Framework (Google Test)
-//
-// This header file defines the GTEST_OS_* macro.
-// It is separate from gtest-port.h so that custom/gtest-port.h can include it.
-
-#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_ARCH_H_
-#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_ARCH_H_
-
-// Determines the platform on which Google Test is compiled.
-#ifdef __CYGWIN__
-# define GTEST_OS_CYGWIN 1
-#elif defined __SYMBIAN32__
-# define GTEST_OS_SYMBIAN 1
-#elif defined _WIN32
-# define GTEST_OS_WINDOWS 1
-# ifdef _WIN32_WCE
-#  define GTEST_OS_WINDOWS_MOBILE 1
-# elif defined(__MINGW__) || defined(__MINGW32__)
-#  define GTEST_OS_WINDOWS_MINGW 1
-# elif defined(WINAPI_FAMILY)
-#  include <winapifamily.h>
-#  if WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_DESKTOP)
-#   define GTEST_OS_WINDOWS_DESKTOP 1
-#  elif WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_PHONE_APP)
-#   define GTEST_OS_WINDOWS_PHONE 1
-#  elif WINAPI_FAMILY_PARTITION(WINAPI_PARTITION_APP)
-#   define GTEST_OS_WINDOWS_RT 1
-#  else
-    // WINAPI_FAMILY defined but no known partition matched.
-    // Default to desktop.
-#   define GTEST_OS_WINDOWS_DESKTOP 1
-#  endif
-# else
-#  define GTEST_OS_WINDOWS_DESKTOP 1
-# endif  // _WIN32_WCE
-#elif defined __APPLE__
-# define GTEST_OS_MAC 1
-# if TARGET_OS_IPHONE
-#  define GTEST_OS_IOS 1
-# endif
-#elif defined __FreeBSD__
-# define GTEST_OS_FREEBSD 1
-#elif defined __linux__
-# define GTEST_OS_LINUX 1
-# if defined __ANDROID__
-#  define GTEST_OS_LINUX_ANDROID 1
-# endif
-#elif defined __MVS__
-# define GTEST_OS_ZOS 1
-#elif defined(__sun) && defined(__SVR4)
-# define GTEST_OS_SOLARIS 1
-#elif defined(_AIX)
-# define GTEST_OS_AIX 1
-#elif defined(__hpux)
-# define GTEST_OS_HPUX 1
-#elif defined __native_client__
-# define GTEST_OS_NACL 1
-#elif defined __OpenBSD__
-# define GTEST_OS_OPENBSD 1
-#elif defined __QNX__
-# define GTEST_OS_QNX 1
-#endif  // __CYGWIN__
-
-#endif  // GTEST_INCLUDE_GTEST_INTERNAL_GTEST_PORT_ARCH_H_
-// Copyright 2015, Google Inc.
-// All rights reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Injection point for custom user configurations.
-// The following macros can be defined:
-//
-//   Flag related macros:
-//     GTEST_FLAG(flag_name)
-//     GTEST_USE_OWN_FLAGFILE_FLAG_  - Define to 0 when the system provides its
-//                                     own flagfile flag parsing.
-//     GTEST_DECLARE_bool_(name)
-//     GTEST_DECLARE_int32_(name)
-//     GTEST_DECLARE_string_(name)
-//     GTEST_DEFINE_bool_(name, default_val, doc)
-//     GTEST_DEFINE_int32_(name, default_val, doc)
-//     GTEST_DEFINE_string_(name, default_val, doc)
-//
-//   Test filtering:
-//     GTEST_TEST_FILTER_ENV_VAR_ - The name of an environment variable that
-//                                  will be used if --GTEST_FLAG(test_filter)
-//                                  is not provided.
-//
-//   Logging:
-//     GTEST_LOG_(severity)
-//     GTEST_CHECK_(condition)
-//     Functions LogToStderr() and FlushInfoLog() have to be provided too.
-//
-//   Threading:
-//     GTEST_HAS_NOTIFICATION_ - Enabled if Notification is already provided.
-//     GTEST_HAS_MUTEX_AND_THREAD_LOCAL_ - Enabled if Mutex and ThreadLocal are
-//                                         already provided.
-//     Must also provide GTEST_DECLARE_STATIC_MUTEX_(mutex) and
-//     GTEST_DEFINE_STATIC_MUTEX_(mutex)
-//
-//     GTEST_EXCLUSIVE_LOCK_REQUIRED_(locks)
-//     GTEST_LOCK_EXCLUDED_(locks)
-//
-// ** Custom implementation starts here **
-
-#ifndef GTEST_INCLUDE_GTEST_INTERNAL_CUSTOM_GTEST_PORT_H_
-#define GTEST_INCLUDE_GTEST_INTERNAL_CUSTOM_GTEST_PORT_H_
-
-#endif  // GTEST_INCLUDE_GTEST_INTERNAL_CUSTOM_GTEST_PORT_H_
-
-#if !defined(GTEST_DEV_EMAIL_)
-# define GTEST_DEV_EMAIL_ "googletestframework@@googlegroups.com"
-# define GTEST_FLAG_PREFIX_ "gtest_"
-# define GTEST_FLAG_PREFIX_DASH_ "gtest-"
-# define GTEST_FLAG_PREFIX_UPPER_ "GTEST_"
-# define GTEST_NAME_ "Google Test"
-# define GTEST_PROJECT_URL_ "https://github.com/google/googletest/"
-#endif  // !defined(GTEST_DEV_EMAIL_)
-
-#if !defined(GTEST_INIT_GOOGLE_TEST_NAME_)
-# define GTEST_INIT_GOOGLE_TEST_NAME_ "testing::InitGoogleTest"
-#endif  // !defined(GTEST_INIT_GOOGLE_TEST_NAME_)
-
-// Determines the version of gcc that is used to compile this.
-#ifdef __GNUC__
-// 40302 means version 4.3.2.
-# define GTEST_GCC_VER_ \
-    (__GNUC__*10000 + __GNUC_MINOR__*100 + __GNUC_PATCHLEVEL__)
-#endif  // __GNUC__
-
-// Macros for disabling Microsoft Visual C++ warnings.
-//
-//   GTEST_DISABLE_MSC_WARNINGS_PUSH_(4800 4385)
-//   /* code that triggers warnings C4800 and C4385 */
-//   GTEST_DISABLE_MSC_WARNINGS_POP_()
-#if _MSC_VER >= 1500
-# define GTEST_DISABLE_MSC_WARNINGS_PUSH_(warnings) \
-    __pragma(warning(push))                        \
-    __pragma(warning(disable: warnings))
-# define GTEST_DISABLE_MSC_WARNINGS_POP_()          \
-    __pragma(warning(pop))
-#else
-// Older versions of MSVC don't have __pragma.
-# define GTEST_DISABLE_MSC_WARNINGS_PUSH_(warnings)
-# define GTEST_DISABLE_MSC_WARNINGS_POP_()
-#endif
-
-#ifndef GTEST_LANG_CXX11
-// gcc and clang define __GXX_EXPERIMENTAL_CXX0X__ when
-// -std={c,gnu}++{0x,11} is passed.  The C++11 standard specifies a
-// value for __cplusplus, and recent versions of clang, gcc, and
-// probably other compilers set that too in C++11 mode.
-# if __GXX_EXPERIMENTAL_CXX0X__ || __cplusplus >= 201103L
-// Compiling in at least C++11 mode.
-#  define GTEST_LANG_CXX11 1
-# else
-#  define GTEST_LANG_CXX11 0
-# endif
-#endif
-
-// Distinct from C++11 language support, some environments don't provide
-// proper C++11 library support. Notably, it's possible to build in
-// C++11 mode when targeting Mac OS X 10.6, which has an old libstdc++
-// with no C++11 support.
-//
-// libstdc++ has sufficient C++11 support as of GCC 4.6.0, __GLIBCXX__
-// 20110325, but maintenance releases in the 4.4 and 4.5 series followed
-// this date, so check for those versions by their date stamps.
-// https://gcc.gnu.org/onlinedocs/libstdc++/manual/abi.html#abi.versioning
-#if GTEST_LANG_CXX11 && \
-    (!defined(__GLIBCXX__) || ( \
-        __GLIBCXX__ >= 20110325ul &&  /* GCC >= 4.6.0 */ \
-        /* Blacklist of patch releases of older branches: */ \
-        __GLIBCXX__ != 20110416ul &&  /* GCC 4.4.6 */ \
-        __GLIBCXX__ != 20120313ul &&  /* GCC 4.4.7 */ \
-        __GLIBCXX__ != 20110428ul &&  /* GCC 4.5.3 */ \
-        __GLIBCXX__ != 20120702ul))   /* GCC 4.5.4 */
-# define GTEST_STDLIB_CXX11 1
-#endif
-
-// Only use C++11 library features if the library provides them.
-#if GTEST_STDLIB_CXX11
-# define GTEST_HAS_STD_BEGIN_AND_END_ 1
-# define GTEST_HAS_STD_FORWARD_LIST_ 1
-# define GTEST_HAS_STD_FUNCTION_ 1
-# define GTEST_HAS_STD_INITIALIZER_LIST_ 1
-# define GTEST_HAS_STD_MOVE_ 1
-# define GTEST_HAS_STD_SHARED_PTR_ 1
-# define GTEST_HAS_STD_TYPE_TRAITS_ 1
-# define GTEST_HAS_STD_UNIQUE_PTR_ 1
-#endif
-
-// C++11 specifies that <tuple> provides std::tuple.
-// Some platforms still might not have it, however.
-#if GTEST_LANG_CXX11
-# define GTEST_HAS_STD_TUPLE_ 1
-# if defined(__clang__)
-// Inspired by http://clang.llvm.org/docs/LanguageExtensions.html#__has_include
-#  if defined(__has_include) && !__has_include(<tuple>)
-#   undef GTEST_HAS_STD_TUPLE_
-#  endif
-# elif defined(_MSC_VER)
-// Inspired by boost/config/stdlib/dinkumware.hpp
-#  if defined(_CPPLIB_VER) && _CPPLIB_VER < 520
-#   undef GTEST_HAS_STD_TUPLE_
-#  endif
-# elif defined(__GLIBCXX__)
-// Inspired by boost/config/stdlib/libstdcpp3.hpp,
-// http://gcc.gnu.org/gcc-4.2/changes.html and
-// http://gcc.gnu.org/onlinedocs/libstdc++/manual/bk01pt01ch01.html#manual.intro.status.standard.200x
-#  if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 2)
-#   undef GTEST_HAS_STD_TUPLE_
-#  endif
-# endif
-#endif
-
-// Brings in definitions for functions used in the testing::internal::posix
-// namespace (read, write, close, chdir, isatty, stat). We do not currently
-// use them on Windows Mobile.
-#if GTEST_OS_WINDOWS
-# if !GTEST_OS_WINDOWS_MOBILE
-#  include <direct.h>
-#  include <io.h>
-# endif
-// In order to avoid having to include <windows.h>, use forward declaration
-// assuming CRITICAL_SECTION is a typedef of _RTL_CRITICAL_SECTION.
-// This assumption is verified by
-// WindowsTypesTest.CRITICAL_SECTIONIs_RTL_CRITICAL_SECTION.
-struct _RTL_CRITICAL_SECTION;
-#else
-// This assumes that non-Windows OSes provide unistd.h. For OSes where this
-// is not the case, we need to include headers that provide the functions
-// mentioned above.
-# include <unistd.h>
-# include <strings.h>
-#endif  // GTEST_OS_WINDOWS
-
-#if GTEST_OS_LINUX_ANDROID
-// Used to define __ANDROID_API__ matching the target NDK API level.
-#  include <android/api-level.h>  // NOLINT
-#endif
-
-// Defines this to true iff Google Test can use POSIX regular expressions.
-#ifndef GTEST_HAS_POSIX_RE
-# if GTEST_OS_LINUX_ANDROID
-// On Android, <regex.h> is only available starting with Gingerbread.
-#  define GTEST_HAS_POSIX_RE (__ANDROID_API__ >= 9)
-# else
-#  define GTEST_HAS_POSIX_RE (!GTEST_OS_WINDOWS)
-# endif
-#endif
-
-#if GTEST_USES_PCRE
-// The appropriate headers have already been included.
-
-#elif GTEST_HAS_POSIX_RE
-
-// On some platforms, <regex.h> needs someone to define size_t, and
-// won't compile otherwise.  We can #include it here as we already
-// included <stdlib.h>, which is guaranteed to define size_t through
-// <stddef.h>.
-# include <regex.h>  // NOLINT
-
-# define GTEST_USES_POSIX_RE 1
-
-#elif GTEST_OS_WINDOWS
-
-// <regex.h> is not available on Windows.  Use our own simple regex
-// implementation instead.
-# define GTEST_USES_SIMPLE_RE 1
-
-#else
-
-// <regex.h> may not be available on this platform.  Use our own
-// simple regex implementation instead.
-# define GTEST_USES_SIMPLE_RE 1
-
-#endif  // GTEST_USES_PCRE
-
-#ifndef GTEST_HAS_EXCEPTIONS
-// The user didn't tell us whether exceptions are enabled, so we need
-// to figure it out.
-# if defined(_MSC_VER) || defined(__BORLANDC__)
-// MSVC's and C++Builder's implementations of the STL use the _HAS_EXCEPTIONS
-// macro to enable exceptions, so we'll do the same.
-// Assumes that exceptions are enabled by default.
-#  ifndef _HAS_EXCEPTIONS
-#   define _HAS_EXCEPTIONS 1
-#  endif  // _HAS_EXCEPTIONS
-#  define GTEST_HAS_EXCEPTIONS _HAS_EXCEPTIONS
-# elif defined(__clang__)
-// clang defines __EXCEPTIONS iff exceptions are enabled before clang 220714,
-// but iff cleanups are enabled after that. In Obj-C++ files, there can be
-// cleanups for ObjC exceptions which also need cleanups, even if C++ exceptions
-// are disabled. clang has __has_feature(cxx_exceptions) which checks for C++
-// exceptions starting at clang r206352, but which checked for cleanups prior to
-// that. To reliably check for C++ exception availability with clang, check for
-// __EXCEPTIONS && __has_feature(cxx_exceptions).
-#  define GTEST_HAS_EXCEPTIONS (__EXCEPTIONS && __has_feature(cxx_exceptions))
-# elif defined(__GNUC__) && __EXCEPTIONS
-// gcc defines __EXCEPTIONS to 1 iff exceptions are enabled.
-#  define GTEST_HAS_EXCEPTIONS 1
-# elif defined(__SUNPRO_CC)
-// Sun Pro CC supports exceptions.  However, there is no compile-time way of
-// detecting whether they are enabled or not.  Therefore, we assume that
-// they are enabled unless the user tells us otherwise.
-#  define GTEST_HAS_EXCEPTIONS 1
-# elif defined(__IBMCPP__) && __EXCEPTIONS
-// xlC defines __EXCEPTIONS to 1 iff exceptions are enabled.
-#  define GTEST_HAS_EXCEPTIONS 1
-# elif defined(__HP_aCC)
-// Exception handling is in effect by default in HP aCC compiler. It has to
-// be turned of by +noeh compiler option if desired.
-#  define GTEST_HAS_EXCEPTIONS 1
-# else
-// For other compilers, we assume exceptions are disabled to be
-// conservative.
-#  define GTEST_HAS_EXCEPTIONS 0
-# endif  // defined(_MSC_VER) || defined(__BORLANDC__)
-#endif  // GTEST_HAS_EXCEPTIONS
-
-#if !defined(GTEST_HAS_STD_STRING)
-// Even though we don't use this macro any longer, we keep it in case
-// some clients still depend on it.
-# define GTEST_HAS_STD_STRING 1
-#elif !GTEST_HAS_STD_STRING
-// The user told us that ::std::string isn't available.
-# error "Google Test cannot be used where ::std::string isn't available."
-#endif  // !defined(GTEST_HAS_STD_STRING)
-
-#ifndef GTEST_HAS_GLOBAL_STRING
-// The user didn't tell us whether ::string is available, so we need
-// to figure it out.
-
-# define GTEST_HAS_GLOBAL_STRING 0
-
-#endif  // GTEST_HAS_GLOBAL_STRING
-
-#ifndef GTEST_HAS_STD_WSTRING
-// The user didn't tell us whether ::std::wstring is available, so we need
-// to figure it out.
-// TODO(wan@google.com): uses autoconf to detect whether ::std::wstring
-//   is available.
-
-// Cygwin 1.7 and below doesn't support ::std::wstring.
-// Solaris' libc++ doesn't support it either.  Android has
-// no support for it at least as recent as Froyo (2.2).
-# define GTEST_HAS_STD_WSTRING \
-    (!(GTEST_OS_LINUX_ANDROID || GTEST_OS_CYGWIN || GTEST_OS_SOLARIS))
-
-#endif  // GTEST_HAS_STD_WSTRING
-
-#ifndef GTEST_HAS_GLOBAL_WSTRING
-// The user didn't tell us whether ::wstring is available, so we need
-// to figure it out.
-# define GTEST_HAS_GLOBAL_WSTRING \
-    (GTEST_HAS_STD_WSTRING && GTEST_HAS_GLOBAL_STRING)
-#endif  // GTEST_HAS_GLOBAL_WSTRING
-
-// Determines whether RTTI is available.
-#ifndef GTEST_HAS_RTTI
-// The user didn't tell us whether RTTI is enabled, so we need to
-// figure it out.
-
-# ifdef _MSC_VER
-
-#  ifdef _CPPRTTI  // MSVC defines this macro iff RTTI is enabled.
-#   define GTEST_HAS_RTTI 1
-#  else
-#   define GTEST_HAS_RTTI 0
-#  endif
-
-// Starting with version 4.3.2, gcc defines __GXX_RTTI iff RTTI is enabled.
-# elif defined(__GNUC__) && (GTEST_GCC_VER_ >= 40302)
-
-#  ifdef __GXX_RTTI
-// When building against STLport with the Android NDK and with
-// -frtti -fno-exceptions, the build fails at link time with undefined
-// references to __cxa_bad_typeid. Note sure if STL or toolchain bug,
-// so disable RTTI when detected.
-#   if GTEST_OS_LINUX_ANDROID && defined(_STLPORT_MAJOR) && \
-       !defined(__EXCEPTIONS)
-#    define GTEST_HAS_RTTI 0
-#   else
-#    define GTEST_HAS_RTTI 1
-#   endif  // GTEST_OS_LINUX_ANDROID && __STLPORT_MAJOR && !__EXCEPTIONS
-#  else
-#   define GTEST_HAS_RTTI 0
-#  endif  // __GXX_RTTI
-
-// Clang defines __GXX_RTTI starting with version 3.0, but its manual recommends
-// using has_feature instead. has_feature(cxx_rtti) is supported since 2.7, the
-// first version with C++ support.
-# elif defined(__clang__)
-
-#  define GTEST_HAS_RTTI __has_feature(cxx_rtti)
-
-// Starting with version 9.0 IBM Visual Age defines __RTTI_ALL__ to 1 if
-// both the typeid and dynamic_cast features are present.
-# elif defined(__IBMCPP__) && (__IBMCPP__ >= 900)
-
-#  ifdef __RTTI_ALL__
-#   define GTEST_HAS_RTTI 1
-#  else
-#   define GTEST_HAS_RTTI 0
-#  endif
-
-# else
-
-// For all other compilers, we assume RTTI is enabled.
-#  define GTEST_HAS_RTTI 1
-
-# endif  // _MSC_VER
-
-#endif  // GTEST_HAS_RTTI
-
-// It's this header's responsibility to #include <typeinfo> when RTTI
-// is enabled.
-#if GTEST_HAS_RTTI
-# include <typeinfo>
-#endif
-
-// Determines whether Google Test can use the pthreads library.
-#ifndef GTEST_HAS_PTHREAD
-// The user didn't tell us explicitly, so we make reasonable assumptions about
-// which platforms have pthreads support.
-//
-// To disable threading support in Google Test, add -DGTEST_HAS_PTHREAD=0
-// to your compiler flags.
-# define GTEST_HAS_PTHREAD (GTEST_OS_LINUX || GTEST_OS_MAC || GTEST_OS_HPUX \
-    || GTEST_OS_QNX || GTEST_OS_FREEBSD || GTEST_OS_NACL)
-#endif  // GTEST_HAS_PTHREAD
-
-#if GTEST_HAS_PTHREAD
-// gtest-port.h guarantees to #include <pthread.h> when GTEST_HAS_PTHREAD is
-// true.
-# include <pthread.h>  // NOLINT
-
-// For timespec and nanosleep, used below.
-# include <time.h>  // NOLINT
-#endif
-
-// Determines if hash_map/hash_set are available.
-// Only used for testing against those containers.
-#if !defined(GTEST_HAS_HASH_MAP_)
-# if _MSC_VER
-#  define GTEST_HAS_HASH_MAP_ 1  // Indicates that hash_map is available.
-#  define GTEST_HAS_HASH_SET_ 1  // Indicates that hash_set is available.
-# endif  // _MSC_VER
-#endif  // !defined(GTEST_HAS_HASH_MAP_)
-
-// Determines whether Google Test can use tr1/tuple.  You can define
-// this macro to 0 to prevent Google Test from using tuple (any
-// feature depending on tuple with be disabled in this mode).
-#ifndef GTEST_HAS_TR1_TUPLE
-# if GTEST_OS_LINUX_ANDROID && defined(_STLPORT_MAJOR)
-// STLport, provided with the Android NDK, has neither <tr1/tuple> or <tuple>.
-#  define GTEST_HAS_TR1_TUPLE 0
-# else
-// The user didn't tell us not to do it, so we assume it's OK.
-#  define GTEST_HAS_TR1_TUPLE 1
-# endif
-#endif  // GTEST_HAS_TR1_TUPLE
-
-// Determines whether Google Test's own tr1 tuple implementation
-// should be used.
-#ifndef GTEST_USE_OWN_TR1_TUPLE
-// The user didn't tell us, so we need to figure it out.
-
-// We use our own TR1 tuple if we aren't sure the user has an
-// implementation of it already.  At this time, libstdc++ 4.0.0+ and
-// MSVC 2010 are the only mainstream standard libraries that come
-// with a TR1 tuple implementation.  NVIDIA's CUDA NVCC compiler
-// pretends to be GCC by defining __GNUC__ and friends, but cannot
-// compile GCC's tuple implementation.  MSVC 2008 (9.0) provides TR1
-// tuple in a 323 MB Feature Pack download, which we cannot assume the
-// user has.  QNX's QCC compiler is a modified GCC but it doesn't
-// support TR1 tuple.  libc++ only provides std::tuple, in C++11 mode,
-// and it can be used with some compilers that define __GNUC__.
-# if (defined(__GNUC__) && !defined(__CUDACC__) && (GTEST_GCC_VER_ >= 40000) \
-      && !GTEST_OS_QNX && !defined(_LIBCPP_VERSION)) || _MSC_VER >= 1600
-#  define GTEST_ENV_HAS_TR1_TUPLE_ 1
-# endif
-
-// C++11 specifies that <tuple> provides std::tuple. Use that if gtest is used
-// in C++11 mode and libstdc++ isn't very old (binaries targeting OS X 10.6
-// can build with clang but need to use gcc4.2's libstdc++).
-# if GTEST_LANG_CXX11 && (!defined(__GLIBCXX__) || __GLIBCXX__ > 20110325)
-#  define GTEST_ENV_HAS_STD_TUPLE_ 1
-# endif
-
-# if GTEST_ENV_HAS_TR1_TUPLE_ || GTEST_ENV_HAS_STD_TUPLE_
-#  define GTEST_USE_OWN_TR1_TUPLE 0
-# else
-#  define GTEST_USE_OWN_TR1_TUPLE 1
-# endif
-
-#endif  // GTEST_USE_OWN_TR1_TUPLE
-
-// To avoid conditional compilation everywhere, we make it
-// gtest-port.h's responsibility to #include the header implementing
-// tuple.
-#if GTEST_HAS_STD_TUPLE_
-# include <tuple>  // IWYU pragma: export
-# define GTEST_TUPLE_NAMESPACE_ ::std
-#endif  // GTEST_HAS_STD_TUPLE_
-
-// We include tr1::tuple even if std::tuple is available to define printers for
-// them.
-#if GTEST_HAS_TR1_TUPLE
-# ifndef GTEST_TUPLE_NAMESPACE_
-#  define GTEST_TUPLE_NAMESPACE_ ::std::tr1
-# endif  // GTEST_TUPLE_NAMESPACE_
-
-# if GTEST_USE_OWN_TR1_TUPLE
-// This file was GENERATED by command:
-//     pump.py gtest-tuple.h.pump
-// DO NOT EDIT BY HAND!!!
-
-// Copyright 2009 Google Inc.
-// All Rights Reserved.
-//
-// Redistribution and use in source and binary forms, with or without
-// modification, are permitted provided that the following conditions are
-// met:
-//
-//     * Redistributions of source code must retain the above copyright
-// notice, this list of conditions and the following disclaimer.
-//     * Redistributions in binary form must reproduce the above
-// copyright notice, this list of conditions and the following disclaimer
-// in the documentation and/or other materials provided with the
-// distribution.
-//     * Neither the name of Google Inc. nor the names of its
-// contributors may be used to endorse or promote products derived from
-// this software without specific prior written permission.
-//
-// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-//
-// Author: wan@google.com (Zhanyong Wan)
-
-// Implements a subset of TR1 tuple needed by Google Test and Google Mock.
-
-#ifndef GTEST_INCLUDE_GTEST_INTERNAL_GTEST_TUPLE_H_
-#define GTEST_INCLUDE_GTEST_INTERNAL_GTEST_TUPLE_H_
-
-#include <utility>  // For ::std::pair.
-
-// The compiler used in Symbian has a bug that prevents us from declaring the
-// tuple template as a friend (it complains that tuple is redefined).  This
-// hack bypasses the bug by declaring the members that should otherwise be
-// private as public.
-// Sun Studio versions < 12 also have the above bug.
-#if defined(__SYMBIAN32__) || (defined(__SUNPRO_CC) && __SUNPRO_CC < 0x590)
-# define GTEST_DECLARE_TUPLE_AS_FRIEND_ public:
-#else
-# define GTEST_DECLARE_TUPLE_AS_FRIEND_ \
-    template <GTEST_10_TYPENAMES_(U)> friend class tuple; \
-   private:
-#endif
-
-// Visual Studio 2010, 2012, and 2013 define symbols in std::tr1 that conflict
-// with our own definitions. Therefore using our own tuple does not work on
-// those compilers.
-#if defined(_MSC_VER) && _MSC_VER >= 1600  /* 1600 is Visual Studio 2010 */
-# error "gtest's tuple doesn't compile on Visual Studio 2010 or later. \
-GTEST_USE_OWN_TR1_TUPLE must be set to 0 on those compilers."
-#endif
-
-// GTEST_n_TUPLE_(T) is the type of an n-tuple.
-#define GTEST_0_TUPLE_(T) tuple<>
-#define GTEST_1_TUPLE_(T) tuple<T##0, void, void, void, void, void, void, \
-    void, void, void>
-#define GTEST_2_TUPLE_(T) tuple<T##0, T##1, void, void, void, void, void, \
-    void, void, void>
-#define GTEST_3_TUPLE_(T) tuple<T##0, T##1, T##2, void, void, void, void, \
-    void, void, void>
-#define GTEST_4_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, void, void, void, \
-    void, void, void>
-#define GTEST_5_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, void, void, \
-    void, void, void>
-#define GTEST_6_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, void, \
-    void, void, void>
-#define GTEST_7_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
-    void, void, void>
-#define GTEST_8_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
-    T##7, void, void>
-#define GTEST_9_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
-    T##7, T##8, void>
-#define GTEST_10_TUPLE_(T) tuple<T##0, T##1, T##2, T##3, T##4, T##5, T##6, \
-    T##7, T##8, T##9>
-
-// GTEST_n_TYPENAMES_(T) declares a list of n typenames.
-#define GTEST_0_TYPENAMES_(T)
-#define GTEST_1_TYPENAMES_(T) typename T##0
-#define GTEST_2_TYPENAMES_(T) typename T##0, typename T##1
-#define GTEST_3_TYPENAMES_(T) typename T##0, typename T##1, typename T##2
-#define GTEST_4_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
-    typename T##3
-#define GTEST_5_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
-    typename T##3, typename T##4
-#define GTEST_6_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
-    typename T##3, typename T##4, typename T##5
-#define GTEST_7_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
-    typename T##3, typename T##4, typename T##5, typename T##6
-#define GTEST_8_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
-    typename T##3, typename T##4, typename T##5, typename T##6, typename T##7
-#define GTEST_9_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
-    typename T##3, typename T##4, typename T##5, typename T##6, \
-    typename T##7, typename T##8
-#define GTEST_10_TYPENAMES_(T) typename T##0, typename T##1, typename T##2, \
-    typename T##3, typename T##4, typename T##5, typename T##6, \
-    typename T##7, typename T##8, typename T##9
-
-// In theory, defining stuff in the ::std namespace is undefined
-// behavior.  We can do this as we are playing the role of a standard
-// library vendor.
-namespace std {
-namespace tr1 {
-
-template <typename T0 = void, typename T1 = void, typename T2 = void,
-    typename T3 = void, typename T4 = void, typename T5 = void,
-    typename T6 = void, typename T7 = void, typename T8 = void,
-    typename T9 = void>
-class tuple;
-
-// Anything in namespace gtest_internal is Google Test's INTERNAL
-// IMPLEMENTATION DETAIL and MUST NOT BE USED DIRECTLY in user code.
-namespace gtest_internal {
-
-// ByRef<T>::type is T if T is a reference; otherwise it's const T&.
-template <typename T>
-struct ByRef { typedef const T& type; };  // NOLINT
-template <typename T>
-struct ByRef<T&> { typedef T& type; };  // NOLINT
-
-// A handy wrapper for ByRef.
-#define GTEST_BY_REF_(T) typename ::std::tr1::gtest_internal::ByRef<T>::type
-
-// AddRef<T>::type is T if T is a reference; otherwise it's T&.  This
-// is the same as tr1::add_reference<T>::type.
-template <typename T>
-struct AddRef { typedef T& type; };  // NOLINT
-template <typename T>
-struct AddRef<T&> { typedef T& type; };  // NOLINT
-
-// A handy wrapper for AddRef.
-#define GTEST_ADD_REF_(T) typename ::std::tr1::gtest_internal::AddRef<T>::type
-
-// A helper for implementing get<k>().
-template <int k> class Get;
-
-// A helper for implementing tuple_element<k, T>.  kIndexValid is true
-// iff k < the number of fields in tuple type T.
-template <bool kIndexValid, int kIndex, class Tuple>
-struct TupleElement;
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 0, GTEST_10_TUPLE_(T) > {
-  typedef T0 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 1, GTEST_10_TUPLE_(T) > {
-  typedef T1 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 2, GTEST_10_TUPLE_(T) > {
-  typedef T2 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 3, GTEST_10_TUPLE_(T) > {
-  typedef T3 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 4, GTEST_10_TUPLE_(T) > {
-  typedef T4 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 5, GTEST_10_TUPLE_(T) > {
-  typedef T5 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 6, GTEST_10_TUPLE_(T) > {
-  typedef T6 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 7, GTEST_10_TUPLE_(T) > {
-  typedef T7 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 8, GTEST_10_TUPLE_(T) > {
-  typedef T8 type;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct TupleElement<true, 9, GTEST_10_TUPLE_(T) > {
-  typedef T9 type;
-};
-
-}  // namespace gtest_internal
-
-template <>
-class tuple<> {
- public:
-  tuple() {}
-  tuple(const tuple& /* t */)  {}
-  tuple& operator=(const tuple& /* t */) { return *this; }
-};
-
-template <GTEST_1_TYPENAMES_(T)>
-class GTEST_1_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0) : f0_(f0) {}
-
-  tuple(const tuple& t) : f0_(t.f0_) {}
-
-  template <GTEST_1_TYPENAMES_(U)>
-  tuple(const GTEST_1_TUPLE_(U)& t) : f0_(t.f0_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_1_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_1_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_1_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_1_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    return *this;
-  }
-
-  T0 f0_;
-};
-
-template <GTEST_2_TYPENAMES_(T)>
-class GTEST_2_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1) : f0_(f0),
-      f1_(f1) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_) {}
-
-  template <GTEST_2_TYPENAMES_(U)>
-  tuple(const GTEST_2_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_) {}
-  template <typename U0, typename U1>
-  tuple(const ::std::pair<U0, U1>& p) : f0_(p.first), f1_(p.second) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_2_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_2_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-  template <typename U0, typename U1>
-  tuple& operator=(const ::std::pair<U0, U1>& p) {
-    f0_ = p.first;
-    f1_ = p.second;
-    return *this;
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_2_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_2_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-};
-
-template <GTEST_3_TYPENAMES_(T)>
-class GTEST_3_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2) : f0_(f0), f1_(f1), f2_(f2) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_) {}
-
-  template <GTEST_3_TYPENAMES_(U)>
-  tuple(const GTEST_3_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_3_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_3_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_3_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_3_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-};
-
-template <GTEST_4_TYPENAMES_(T)>
-class GTEST_4_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_(), f3_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3) : f0_(f0), f1_(f1), f2_(f2),
-      f3_(f3) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_) {}
-
-  template <GTEST_4_TYPENAMES_(U)>
-  tuple(const GTEST_4_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
-      f3_(t.f3_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_4_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_4_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_4_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_4_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    f3_ = t.f3_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-  T3 f3_;
-};
-
-template <GTEST_5_TYPENAMES_(T)>
-class GTEST_5_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_(), f3_(), f4_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3,
-      GTEST_BY_REF_(T4) f4) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
-      f4_(t.f4_) {}
-
-  template <GTEST_5_TYPENAMES_(U)>
-  tuple(const GTEST_5_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
-      f3_(t.f3_), f4_(t.f4_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_5_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_5_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_5_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_5_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    f3_ = t.f3_;
-    f4_ = t.f4_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-  T3 f3_;
-  T4 f4_;
-};
-
-template <GTEST_6_TYPENAMES_(T)>
-class GTEST_6_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
-      GTEST_BY_REF_(T5) f5) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4),
-      f5_(f5) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
-      f4_(t.f4_), f5_(t.f5_) {}
-
-  template <GTEST_6_TYPENAMES_(U)>
-  tuple(const GTEST_6_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
-      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_6_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_6_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_6_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_6_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    f3_ = t.f3_;
-    f4_ = t.f4_;
-    f5_ = t.f5_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-  T3 f3_;
-  T4 f4_;
-  T5 f5_;
-};
-
-template <GTEST_7_TYPENAMES_(T)>
-class GTEST_7_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
-      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6) : f0_(f0), f1_(f1), f2_(f2),
-      f3_(f3), f4_(f4), f5_(f5), f6_(f6) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
-      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_) {}
-
-  template <GTEST_7_TYPENAMES_(U)>
-  tuple(const GTEST_7_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
-      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_7_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_7_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_7_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_7_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    f3_ = t.f3_;
-    f4_ = t.f4_;
-    f5_ = t.f5_;
-    f6_ = t.f6_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-  T3 f3_;
-  T4 f4_;
-  T5 f5_;
-  T6 f6_;
-};
-
-template <GTEST_8_TYPENAMES_(T)>
-class GTEST_8_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_(), f7_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
-      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6,
-      GTEST_BY_REF_(T7) f7) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4),
-      f5_(f5), f6_(f6), f7_(f7) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
-      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_) {}
-
-  template <GTEST_8_TYPENAMES_(U)>
-  tuple(const GTEST_8_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
-      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_8_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_8_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_8_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_8_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    f3_ = t.f3_;
-    f4_ = t.f4_;
-    f5_ = t.f5_;
-    f6_ = t.f6_;
-    f7_ = t.f7_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-  T3 f3_;
-  T4 f4_;
-  T5 f5_;
-  T6 f6_;
-  T7 f7_;
-};
-
-template <GTEST_9_TYPENAMES_(T)>
-class GTEST_9_TUPLE_(T) {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_(), f7_(), f8_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
-      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6, GTEST_BY_REF_(T7) f7,
-      GTEST_BY_REF_(T8) f8) : f0_(f0), f1_(f1), f2_(f2), f3_(f3), f4_(f4),
-      f5_(f5), f6_(f6), f7_(f7), f8_(f8) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
-      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_) {}
-
-  template <GTEST_9_TYPENAMES_(U)>
-  tuple(const GTEST_9_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
-      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_9_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_9_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_9_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_9_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    f3_ = t.f3_;
-    f4_ = t.f4_;
-    f5_ = t.f5_;
-    f6_ = t.f6_;
-    f7_ = t.f7_;
-    f8_ = t.f8_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-  T3 f3_;
-  T4 f4_;
-  T5 f5_;
-  T6 f6_;
-  T7 f7_;
-  T8 f8_;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-class tuple {
- public:
-  template <int k> friend class gtest_internal::Get;
-
-  tuple() : f0_(), f1_(), f2_(), f3_(), f4_(), f5_(), f6_(), f7_(), f8_(),
-      f9_() {}
-
-  explicit tuple(GTEST_BY_REF_(T0) f0, GTEST_BY_REF_(T1) f1,
-      GTEST_BY_REF_(T2) f2, GTEST_BY_REF_(T3) f3, GTEST_BY_REF_(T4) f4,
-      GTEST_BY_REF_(T5) f5, GTEST_BY_REF_(T6) f6, GTEST_BY_REF_(T7) f7,
-      GTEST_BY_REF_(T8) f8, GTEST_BY_REF_(T9) f9) : f0_(f0), f1_(f1), f2_(f2),
-      f3_(f3), f4_(f4), f5_(f5), f6_(f6), f7_(f7), f8_(f8), f9_(f9) {}
-
-  tuple(const tuple& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_), f3_(t.f3_),
-      f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_), f9_(t.f9_) {}
-
-  template <GTEST_10_TYPENAMES_(U)>
-  tuple(const GTEST_10_TUPLE_(U)& t) : f0_(t.f0_), f1_(t.f1_), f2_(t.f2_),
-      f3_(t.f3_), f4_(t.f4_), f5_(t.f5_), f6_(t.f6_), f7_(t.f7_), f8_(t.f8_),
-      f9_(t.f9_) {}
-
-  tuple& operator=(const tuple& t) { return CopyFrom(t); }
-
-  template <GTEST_10_TYPENAMES_(U)>
-  tuple& operator=(const GTEST_10_TUPLE_(U)& t) {
-    return CopyFrom(t);
-  }
-
-  GTEST_DECLARE_TUPLE_AS_FRIEND_
-
-  template <GTEST_10_TYPENAMES_(U)>
-  tuple& CopyFrom(const GTEST_10_TUPLE_(U)& t) {
-    f0_ = t.f0_;
-    f1_ = t.f1_;
-    f2_ = t.f2_;
-    f3_ = t.f3_;
-    f4_ = t.f4_;
-    f5_ = t.f5_;
-    f6_ = t.f6_;
-    f7_ = t.f7_;
-    f8_ = t.f8_;
-    f9_ = t.f9_;
-    return *this;
-  }
-
-  T0 f0_;
-  T1 f1_;
-  T2 f2_;
-  T3 f3_;
-  T4 f4_;
-  T5 f5_;
-  T6 f6_;
-  T7 f7_;
-  T8 f8_;
-  T9 f9_;
-};
-
-// 6.1.3.2 Tuple creation functions.
-
-// Known limitations: we don't support passing an
-// std::tr1::reference_wrapper<T> to make_tuple().  And we don't
-// implement tie().
-
-inline tuple<> make_tuple() { return tuple<>(); }
-
-template <GTEST_1_TYPENAMES_(T)>
-inline GTEST_1_TUPLE_(T) make_tuple(const T0& f0) {
-  return GTEST_1_TUPLE_(T)(f0);
-}
-
-template <GTEST_2_TYPENAMES_(T)>
-inline GTEST_2_TUPLE_(T) make_tuple(const T0& f0, const T1& f1) {
-  return GTEST_2_TUPLE_(T)(f0, f1);
-}
-
-template <GTEST_3_TYPENAMES_(T)>
-inline GTEST_3_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2) {
-  return GTEST_3_TUPLE_(T)(f0, f1, f2);
-}
-
-template <GTEST_4_TYPENAMES_(T)>
-inline GTEST_4_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
-    const T3& f3) {
-  return GTEST_4_TUPLE_(T)(f0, f1, f2, f3);
-}
-
-template <GTEST_5_TYPENAMES_(T)>
-inline GTEST_5_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
-    const T3& f3, const T4& f4) {
-  return GTEST_5_TUPLE_(T)(f0, f1, f2, f3, f4);
-}
-
-template <GTEST_6_TYPENAMES_(T)>
-inline GTEST_6_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
-    const T3& f3, const T4& f4, const T5& f5) {
-  return GTEST_6_TUPLE_(T)(f0, f1, f2, f3, f4, f5);
-}
-
-template <GTEST_7_TYPENAMES_(T)>
-inline GTEST_7_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
-    const T3& f3, const T4& f4, const T5& f5, const T6& f6) {
-  return GTEST_7_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6);
-}
-
-template <GTEST_8_TYPENAMES_(T)>
-inline GTEST_8_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
-    const T3& f3, const T4& f4, const T5& f5, const T6& f6, const T7& f7) {
-  return GTEST_8_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6, f7);
-}
-
-template <GTEST_9_TYPENAMES_(T)>
-inline GTEST_9_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
-    const T3& f3, const T4& f4, const T5& f5, const T6& f6, const T7& f7,
-    const T8& f8) {
-  return GTEST_9_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6, f7, f8);
-}
-
-template <GTEST_10_TYPENAMES_(T)>
-inline GTEST_10_TUPLE_(T) make_tuple(const T0& f0, const T1& f1, const T2& f2,
-    const T3& f3, const T4& f4, const T5& f5, const T6& f6, const T7& f7,
-    const T8& f8, const T9& f9) {
-  return GTEST_10_TUPLE_(T)(f0, f1, f2, f3, f4, f5, f6, f7, f8, f9);
-}
-
-// 6.1.3.3 Tuple helper classes.
-
-template <typename Tuple> struct tuple_size;
-
-template <GTEST_0_TYPENAMES_(T)>
-struct tuple_size<GTEST_0_TUPLE_(T) > {
-  static const int value = 0;
-};
-
-template <GTEST_1_TYPENAMES_(T)>
-struct tuple_size<GTEST_1_TUPLE_(T) > {
-  static const int value = 1;
-};
-
-template <GTEST_2_TYPENAMES_(T)>
-struct tuple_size<GTEST_2_TUPLE_(T) > {
-  static const int value = 2;
-};
-
-template <GTEST_3_TYPENAMES_(T)>
-struct tuple_size<GTEST_3_TUPLE_(T) > {
-  static const int value = 3;
-};
-
-template <GTEST_4_TYPENAMES_(T)>
-struct tuple_size<GTEST_4_TUPLE_(T) > {
-  static const int value = 4;
-};
-
-template <GTEST_5_TYPENAMES_(T)>
-struct tuple_size<GTEST_5_TUPLE_(T) > {
-  static const int value = 5;
-};
-
-template <GTEST_6_TYPENAMES_(T)>
-struct tuple_size<GTEST_6_TUPLE_(T) > {
-  static const int value = 6;
-};
-
-template <GTEST_7_TYPENAMES_(T)>
-struct tuple_size<GTEST_7_TUPLE_(T) > {
-  static const int value = 7;
-};
-
-template <GTEST_8_TYPENAMES_(T)>
-struct tuple_size<GTEST_8_TUPLE_(T) > {
-  static const int value = 8;
-};
-
-template <GTEST_9_TYPENAMES_(T)>
-struct tuple_size<GTEST_9_TUPLE_(T) > {
-  static const int value = 9;
-};
-
-template <GTEST_10_TYPENAMES_(T)>
-struct tuple_size<GTEST_10_TUPLE_(T) > {
-  static const int value = 10;
-};
-
-template <int k, class Tuple>
-struct tuple_element {
-  typedef typename gtest_internal::TupleElement<
-      k < (tuple_size<Tuple>::value), k, Tuple>::type type;
-};
-
-#define GTEST_TUPLE_ELEMENT_(k, Tuple) typename tuple_element<k, Tuple >::type
-
-// 6.1.3.4 Element access.
-
-namespace gtest_internal {
-
-template <>
-class Get<0> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(0, Tuple))
-  Field(Tuple& t) { return t.f0_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(0, Tuple))
-  ConstField(const Tuple& t) { return t.f0_; }
-};
-
-template <>
-class Get<1> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(1, Tuple))
-  Field(Tuple& t) { return t.f1_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(1, Tuple))
-  ConstField(const Tuple& t) { return t.f1_; }
-};
-
-template <>
-class Get<2> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(2, Tuple))
-  Field(Tuple& t) { return t.f2_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(2, Tuple))
-  ConstField(const Tuple& t) { return t.f2_; }
-};
-
-template <>
-class Get<3> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(3, Tuple))
-  Field(Tuple& t) { return t.f3_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(3, Tuple))
-  ConstField(const Tuple& t) { return t.f3_; }
-};
-
-template <>
-class Get<4> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(4, Tuple))
-  Field(Tuple& t) { return t.f4_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(4, Tuple))
-  ConstField(const Tuple& t) { return t.f4_; }
-};
-
-template <>
-class Get<5> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(5, Tuple))
-  Field(Tuple& t) { return t.f5_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(5, Tuple))
-  ConstField(const Tuple& t) { return t.f5_; }
-};
-
-template <>
-class Get<6> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(6, Tuple))
-  Field(Tuple& t) { return t.f6_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(6, Tuple))
-  ConstField(const Tuple& t) { return t.f6_; }
-};
-
-template <>
-class Get<7> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(7, Tuple))
-  Field(Tuple& t) { return t.f7_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(7, Tuple))
-  ConstField(const Tuple& t) { return t.f7_; }
-};
-
-template <>
-class Get<8> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(8, Tuple))
-  Field(Tuple& t) { return t.f8_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(8, Tuple))
-  ConstField(const Tuple& t) { return t.f8_; }
-};
-
-template <>
-class Get<9> {
- public:
-  template <class Tuple>
-  static GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(9, Tuple))
-  Field(Tuple& t) { return t.f9_; }  // NOLINT
-
-  template <class Tuple>
-  static GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(9, Tuple))
-  ConstField(const Tuple& t) { return t.f9_; }
-};
-
-}  // namespace gtest_internal
-
-template <int k, GTEST_10_TYPENAMES_(T)>
-GTEST_ADD_REF_(GTEST_TUPLE_ELEMENT_(k, GTEST_10_TUPLE_(T)))
-get(GTEST_10_TUPLE_(T)& t) {
-  return gtest_internal::Get<k>::Field(t);
-}
-
-template <int k, GTEST_10_TYPENAMES_(T)>
-GTEST_BY_REF_(GTEST_TUPLE_ELEMENT_(k,  GTEST_10_TUPLE_(T)))
-get(const GTEST_10_TUPLE_(T)& t) {
-  return gtest_internal::Get<k>::ConstField(t);
-}
-
-// 6.1.3.5 Relational operators
-
-// We only implement == and !=, as we don't have a need for the rest yet.
-
-namespace gtest_internal {
-
-// SameSizeTuplePrefixComparator<k, k>::Eq(t1, t2) returns true if the
-// first k fields of t1 equals the first k fields of t2.
-// SameSizeTuplePrefixComparator(k1, k2) would be a compiler error if
-// k1 != k2.
-template <int kSize1, int kSize2>
-struct SameSizeTuplePrefixComparator;
-
-template <>
-struct SameSizeTuplePrefixComparator<0, 0> {
-  template <class Tuple1, class Tuple2>
-  static bool Eq(const Tuple1& /* t1 */, const Tuple2& /* t2 */) {
-    return true;
-  }
-};
-
-template <int k>
-struct SameSizeTuplePrefixComparator<k, k> {
-  template <class Tuple1, class Tuple2>
-  static bool Eq(const Tuple1& t1, const Tuple2& t2) {
-    return SameSizeTuplePrefixComparator<k - 1, k - 1>::Eq(t1, t2) &&
-        ::std::tr1::get<k - 1>(t1) == ::std::tr1::get<k - 1>(t2);
-  }
-};
-
-}  // namespace gtest_internal
-
-template <GTEST_10_TYPENAMES_(T), GTEST_10_TYPENAMES_(U)>
-inline bool operator==(const GTEST_10_TUPLE_(T)& t,
-                       const GTEST_10_TUPLE_(U)& u) {
-  return gtest_internal::SameSizeTuplePrefixComparator<
-      tuple_size<GTEST_10_TUPLE_(T) >::value,
-      tuple_size<GTEST_10_TUPLE_(U) >::value>::Eq(t, u);
-}
-
-template <GTEST_10_TYPENAMES_(T), GTEST_10_TYPENAMES_(U)>
-inline bool operator!=(const GTEST_10_TUPLE_(T)& t,
-                       const GTEST_10_TUPLE_(U)& u) { return !(t == u); }
-
-// 6.1.4 Pairs.
-// Unimplemented.
-
-}  // namespace tr1
-}  // namespace std
-
-#undef GTEST_0_TUPLE_
-#undef GTEST_1_TUPLE_
-#undef GTEST_2_TUPLE_
-#undef GTEST_3_TUPLE_
-#undef GTEST_4_TUPLE_
-#undef GTEST_5_TUPLE_
-#undef GTEST_6_TUPLE_
-#undef GTEST_7_TUPLE_
-#undef GTEST_8_TUPLE_
-#undef GTEST_9_TUPLE_
-#undef GTEST_10_TUPLE_
-
-#undef GTEST_0_TYPENAMES_
-#undef GTEST_1_TYPENAMES_
-#undef GTEST_2_TYPENAMES_
-#undef GTEST_3_TYPENAMES_
-#undef GTEST_4_TYPENAMES_
-#undef GTEST_5_TYPENAMES_
-#undef GTEST_6_TYPENAMES_
-#undef GTEST_7_TYPENAMES_
-#undef GTEST_8_TYPENAMES_
-#undef GTEST_9_TYPENAMES_
-#undef GTEST_10_TYPENAMES_
-
-#undef GTEST_DECLARE_TUPLE_AS_FRIEND_
-#undef GTEST_BY_REF_
-#undef GTEST_ADD_REF_
-#undef GTEST_TUPLE_ELEMENT_
-
-#endif  // GTEST_INCLUDE_GTEST_INTERNAL_GTEST_TUPLE_H_
-# elif GTEST_ENV_HAS_STD_TUPLE_
-#  include <tuple>
-// C++11 puts its tuple into the ::std namespace rather than
-// ::std::tr1.  gtest expects tuple to live in ::std::tr1, so put it there.
-// This causes undefined behavior, but supported compilers react in
-// the way we intend.
-namespace std {
-namespace tr1 {
-using ::std::get;
-using ::std::make_tuple;
-using ::std::tuple;
-using ::std::tuple_element;
-using ::std::tuple_size;
-}
-}
-
-# elif GTEST_OS_SYMBIAN
-
-// On Symbian, BOOST_HAS_TR1_TUPLE causes Boost's TR1 tuple library to
-// use STLport's tuple implementation, which unfortunately doesn't
-// work as the copy of STLport distributed with Symbian is incomplete.
-// By making sure BOOST_HAS_TR1_TUPLE is undefined, we force Boost to
-// use its own tuple implementation.
-#  ifdef BOOST_HAS_TR1_TUPLE
-#   undef BOOST_HAS_TR1_TUPLE
-#  endif  // BOOST_HAS_TR1_TUPLE
-
-// This prevents <boost/tr1/detail/config.hpp>, which defines
-// BOOST_HAS_TR1_TUPLE, from being #included by Boost's <tuple>.
-#  define BOOST_TR1_DETAIL_CONFIG_HPP_INCLUDED
-#  include <tuple>  // IWYU pragma: export  // NOLINT
-
-# elif defined(__GNUC__) && (GTEST_GCC_VER_ >= 40000)
-// GCC 4.0+ implements tr1/tuple in the <tr1/tuple> header.  This does
-// not conform to the TR1 spec, which requires the header to be <tuple>.
-
-#  if !GTEST_HAS_RTTI && GTEST_GCC_VER_ < 40302
-// Until version 4.3.2, gcc has a bug that causes <tr1/functional>,
-// which is #included by <tr1/tuple>, to not compile when RTTI is
-// disabled.  _TR1_FUNCTIONAL is the header guard for
-// <tr1/functional>.  Hence the following #define is a hack to prevent
-// <tr1/functional> from being included.
-#   define _TR1_FUNCTIONAL 1
-#   include <tr1/tuple>
-#   undef _TR1_FUNCTIONAL  // Allows the user to #include
-                        // <tr1/functional> if he chooses to.
-#  else
-#   include <tr1/tuple>  // NOLINT
-#  endif  // !GTEST_HAS_RTTI && GTEST_GCC_VER_ < 40302
-
-# else
-// If the compiler is not GCC 4.0+, we assume the user is using a
-// spec-conforming TR1 implementation.
-#  include <tuple>  // IWYU pragma: export  // NOLINT
-# endif  // GTEST_USE_OWN_TR1_TUPLE
-
-#endif  // GTEST_HAS_TR1_TUPLE
-
-// Determines whether clone(2) is supported.
-// Usually it will only be available on Linux, excluding
-// Linux on the Itanium architecture.
-// Also see http://linux.die.net/man/2/clone.
-#ifndef GTEST_HAS_CLONE
-// The user didn't tell us, so we need to figure it out.
-
-# if GTEST_OS_LINUX && !defined(__ia64__)
-#  if GTEST_OS_LINUX_ANDROID
-// On Android, clone() is only available on ARM starting with Gingerbread.
-#    if defined(__arm__) && __ANDROID_API__ >= 9
-#     define GTEST_HAS_CLONE 1
-#    else
-#     define GTEST_HAS_CLONE 0
-#    endif
-#  else
-#   define GTEST_HAS_CLONE 1
-#  endif
-# else
-#  define GTEST_HAS_CLONE 0
-# endif  // GTEST_OS_LINUX && !defined(__ia64__)
-
-#endif  // GTEST_HAS_CLONE
-
-// Determines whether to support stream redirection. This is used to test
-// output correctness and to implement death tests.
-#ifndef GTEST_HAS_STREAM_REDIRECTION
-// By default, we assume that stream redirection is supported on all
-// platforms except known mobile ones.
-# if GTEST_OS_WINDOWS_MOBILE || GTEST_OS_SYMBIAN || \
-    GTEST_OS_WINDOWS_PHONE || GTEST_OS_WINDOWS_RT
-#  define GTEST_HAS_STREAM_REDIRECTION 0
-# else
-#  define GTEST_HAS_STREAM_REDIRECTION 1
-# endif  // !GTEST_OS_WINDOWS_MOBILE && !GTEST_OS_SYMBIAN
-#endif  // GTEST_HAS_STREAM_REDIRECTION
-
-// Determines whether to support death tests.
-// Google Test does not support death tests for VC 7.1 and earlier as
-// abort() in a VC 7.1 application compiled as GUI in debug config
-// pops up a dialog window that cannot be suppressed programmatically.
-#if (GTEST_OS_LINUX || GTEST_OS_CYGWIN || GTEST_OS_SOLARIS || \
-     (GTEST_OS_MAC && !GTEST_OS_IOS) || \
-     (GTEST_OS_WINDOWS_DESKTOP && _MSC_VER >= 1400) || \
-     GTEST_OS_WINDOWS_MINGW || GTEST_OS_AIX || GTEST_OS_HPUX || \
-     GTEST_OS_OPENBSD || GTEST_OS_QNX || GTEST_OS_FREEBSD)
-# define GTEST_HAS_DEATH_TEST 1
-#endif
-
-// We don't support MSVC 7.1 with exceptions disabled now.  Therefore
-// all the compilers we care about are adequate for supporting
-// value-parameterized tests.
-#define GTEST_HAS_PARAM_TEST 1
-
-// Determines whether to support type-driven tests.
-
-// Typed tests need <typeinfo> and variadic macros, which GCC, VC++ 8.0,
-// Sun Pro CC, IBM Visual Age, and HP aCC support.
-#if defined(__GNUC__) || (_MSC_VER >= 1400) || defined(__SUNPRO_CC) || \
-    defined(__IBMCPP__) || defined(__HP_aCC)
-# define GTEST_HAS_TYPED_TEST 1
-# define GTEST_HAS_TYPED_TEST_P 1
-#endif
-
-// Determines whether to support Combine(). This only makes sense when
-// value-parameterized tests are enabled.  The implementation doesn't
-// work on Sun Studio since it doesn't understand templated conversion
-// operators.
-#if GTEST_HAS_PARAM_TEST && GTEST_HAS_TR1_TUPLE && !defined(__SUNPRO_CC)
-# define GTEST_HAS_COMBINE 1
-#endif
-
-// Determines whether the system compiler uses UTF-16 for encoding wide strings.
-#define GTEST_WIDE_STRING_USES_UTF16_ \
-    (GTEST_OS_WINDOWS || GTEST_OS_CYGWIN || GTEST_OS_SYMBIAN || GTEST_OS_AIX)
-
-// Determines whether test results can be streamed to a socket.
-#if GTEST_OS_LINUX
-# define GTEST_CAN_STREAM_RESULTS_ 1
-#endif
-
-// Defines some utility macros.
-
-// The GNU compiler emits a warning if nested "if" statements are followed by
-// an "else" statement and braces are not used to explicitly disambiguate the
-// "else" binding.  This leads to problems with code like:
-//
-//   if (gate)
-//     ASSERT_*(condition) << "Some message";
-//
-// The "switch (0) case 0:" idiom is used to suppress this.
-#ifdef __INTEL_COMPILER
-# define GTEST_AMBIGUOUS_ELSE_BLOCKER_
-#else
-# define GTEST_AMBIGUOUS_ELSE_BLOCKER_ switch (0) case 0: default:  // NOLINT
-#endif
-
-// Use this annotation at the end of a struct/class definition to
-// prevent the compiler from optimizing away instances that are never
-// used.  This is useful when all interesting logic happens inside the
-// c'tor and / or d'tor.  Example:
-//
-//   struct Foo {
-//     Foo() { ... }
-//   } GTEST_ATTRIBUTE_UNUSED_;
-//
-// Also use it after a variable or parameter declaration to tell the
-// compiler the variable/parameter does not have to be used.
-#if defined(__GNUC__) && !defined(COMPILER_ICC)
-# define GTEST_ATTRIBUTE_UNUSED_ __attribute__ ((unused))
-#elif defined(__clang__)
-# if __has_attribute(unused)
-#  define GTEST_ATTRIBUTE_UNUSED_ __attribute__ ((unused))
-# endif
-#endif
-#ifndef GTEST_ATTRIBUTE_UNUSED_
-# define GTEST_ATTRIBUTE_UNUSED_
-#endif
-
-// A macro to disallow operator=
-// This should be used in the private: declarations for a class.
-#define GTEST_DISALLOW_ASSIGN_(type)\
-  void operator=(type const &)
-
-// A macro to disallow copy constructor and operator=
-// This should be used in the private: declarations for a class.
-#define GTEST_DISALLOW_COPY_AND_ASSIGN_(type)\
-  type(type const &);\
-  GTEST_DISALLOW_ASSIGN_(type)
-
-// Tell the compiler to warn about unused return values for functions declared
-// with this macro.  The macro should be used on function declarations
-// following the argument list:
-//
-//   Sprocket* AllocateSprocket() GTEST_MUST_USE_RESULT_;
-#if defined(__GNUC__) && (GTEST_GCC_VER_ >= 30400) && !defined(COMPILER_ICC)
-# define GTEST_MUST_USE_RESULT_ __attribute__ ((warn_unused_result))
-#else
-# define GTEST_MUST_USE_RESULT_
-#endif  // __GNUC__ && (GTEST_GCC_VER_ >= 30400) && !COMPILER_ICC
-
-// MS C++ compiler emits warning when a conditional expression is compile time
-// constant. In some contexts this warning is false positive and needs to be
-// suppressed. Use the following two macros in such cases:
-//
-// GTEST_INTENTIONAL_CONST_COND_PUSH_()
-// while (true) {
-// GTEST_INTENTIONAL_CONST_COND_POP_()
-// }
-# define GTEST_INTENTIONAL_CONST_COND_PUSH_() \
-    GTEST_DISABLE_MSC_WARNINGS_PUSH_(4127)
-# define GTEST_INTENTIONAL_CONST_COND_POP_() \
-    GTEST_DISABLE_MSC_WARNINGS_POP_()
-
-// Determine whether the compiler supports Microsoft's Structured Exception
-// Handling.  This is supported by several Windows compilers but generally
-// does not exist on any other system.
-#ifndef GTEST_HAS_SEH
-// The user didn't tell us, so we need to figure it out.
-
-# if defined(_MSC_VER) || defined(__BORLANDC__)
-// These two compilers are known to support SEH.
-#  define GTEST_HAS_SEH 1
-# else
-// Assume no SEH.
-#  define GTEST_HAS_SEH 0
-# endif
-
-#define GTEST_IS_THREADSAFE \
-    (GTEST_HAS_MUTEX_AND_THREAD_LOCAL_ \
-     || (GTEST_OS_WINDOWS && !GTEST_OS_WINDOWS_PHONE && !GTEST_OS_WINDOWS_RT) \
-     || GTEST_HAS_PTHREAD)
-
-#endif  // GTEST_HAS_SEH
-
-#ifdef _MSC_VER
-# if GTEST_LINKED_AS_SHARED_LIBRARY
-#  define GTEST_API_ __declspec(dllimport)
-# elif GTEST_CREATE_SHARED_LIBRARY
-#  define GTEST_API_ __declspec(dllexport)
-# endif
-#elif __GNUC__ >= 4 || defined(__clang__)
-# define GTEST_API_ __attribute__((visibility ("default")))
-#endif // _MSC_VER
-
-#ifndef GTEST_API_
-# define GTEST_API_
-#endif
-
-#ifdef __GNUC__
-// Ask the compiler to never inline a given function.
-# define GTEST_NO_INLINE_ __attribute__((noinline))
-#else
-# define GTEST_NO_INLINE_
-#endif
-
-// _LIBCPP_VERSION is defined by the libc++ library from the LLVM project.
-#if defined(__GLIBCXX__) || defined(_LIBCPP_VERSION)
-# define GTEST_HAS_CXXABI_H_ 1
-#else
-# define GTEST_HAS_CXXABI_H_ 0
-#endif
-
-// A function level attribute to disable checking for use of uninitialized
-// memory when built with MemorySanitizer.
-#if defined(__clang__)
-# if __has_feature(memory_sanitizer)
-#  define GTEST_ATTRIBUTE_NO_SANITIZE_MEMORY_ \
-       __attribute__((no_sanitize_memory))
-# else
-#  define GTEST_ATTRIBUTE_NO_SANITIZE_MEMORY_
-# endif  // __has_feature(memory_sanitizer)
-#else
-# define GTEST_ATTRIBUTE_NO_SANITIZE_MEMORY_
-#endif  // __clang__
-
-// A function level attribute to disable AddressSanitizer instrumentation.
-#if defined(__clang__)
-# if __has_feature(address_sanitizer)
-#  define GTEST_ATTRIBUTE_NO_SANITIZE_ADDRESS_ \
-       __attribute__((no_sanitize_address))
-# else
-#  define GTEST_ATTRIBUTE_NO_SANITIZE_ADDRESS_
-# endif  // __has_feature(address_sanitizer)
-#else
-# define GTEST_ATTRIBUTE_NO_SANITIZE_ADDRESS_
-#endif  // __clang__
-
-// A function level attribute to disable ThreadSanitizer instrumentation.
-#if defined(__clang__)
-# if __has_feature(thread_sanitizer)
-#  define GTEST_ATTRIBUTE_NO_SANITIZE_THREAD_ \
-       __attribute__((no_sanitize_thread))
-# else
-#  define GTEST_ATTRIBUTE_NO_SANITIZE_THREAD_
-# endif  // __has_feature(thread_sanitizer)
-#else
-# define GTEST_ATTRIBUTE_NO_SANITIZE_THREAD_
-#endif  // __clang__
-
-namespace testing {
-
-class Message;
-
-#if defined(GTEST_TUPLE_NAMESPACE_)
-// Import tuple and friends into the ::testing namespace.
-// It is part of our interface, having them in ::testing allows us to change
-// their types as needed.
-using GTEST_TUPLE_NAMESPACE_::get;
-using GTEST_TUPLE_NAMESPACE_::make_tuple;
-using GTEST_TUPLE_NAMESPACE_::tuple;
-using GTEST_TUPLE_NAMESPACE_::tuple_size;
-using GTEST_TUPLE_NAMESPACE_::tuple_element;
-#endif  // defined(GTEST_TUPLE_NAMESPACE_)
-
-namespace internal {
-
-// A secret type that Google Test users don't know about.  It has no
-// definition on purpose.  Therefore it's impossible to create a
-// Secret object, which is what we want.
-class Secret;
-
-// The GTEST_COMPILE_ASSERT_ macro can be used to verify that a compile time
-// expression is true. For example, you could use it to verify the
-// size of a static array:
-//
-//   GTEST_COMPILE_ASSERT_(GTEST_ARRAY_SIZE_(names) == NUM_NAMES,
-//                         names_incorrect_size);
-//
-// or to make sure a struct is smaller than a certain size:
-//
-//   GTEST_COMPILE_ASSERT_(sizeof(foo) < 128, foo_too_large);
-//
-// The second argument to the macro is the name of the variable. If
-// the expression is false, most compilers will issue a warning/error
-// containing the name of the variable.
-
-#if GTEST_LANG_CXX11
-# define GTEST_COMPILE_ASSERT_(expr, msg) static_assert(expr, #msg)
-#else  // !GTEST_LANG_CXX11
-template <bool>
-  struct CompileAssert {
-};
-
-# define GTEST_COMPILE_ASSERT_(expr, msg) \
-  typedef ::testing::internal::CompileAssert<(static_cast<bool>(expr))> \
-      msg[static_cast<bool>(expr) ? 1 : -1] GTEST_ATTRIBUTE_UNUSED_
-#endif  // !GTEST_LANG_CXX11
-
-// Implementation details of GTEST_COMPILE_ASSERT_:
-//
-// (In C++11, we simply use static_assert instead of the following)
-//
-// - GTEST_COMPILE_ASSERT_ works by defining an array type that has -1
-//   elements (and thus is invalid) when the expression is false.
-//
-// - The simpler definition
-//
-//    #define GTEST_COMPILE_ASSERT_(expr, msg) typedef char msg[(expr) ? 1 : -1]
-//
-//   does not work, as gcc supports variable-length arrays whose sizes
-//   are determined at run-time (this is gcc's extension and not part
-//   of the C++ standard).  As a result, gcc fails to reject the
-//   following code with the simple definition:
-//
-//     int foo;
-//     GTEST_COMPILE_ASSERT_(foo, msg); // not supposed to compile as foo is
-//                                      // not a compile-time constant.
-//
-// - By using the type CompileAssert<(bool(expr))>, we ensures that
-//   expr is a compile-time constant.  (Template arguments must be
-//   determined at compile-time.)
-//
-// - The outter parentheses in CompileAssert<(bool(expr))> are necessary
-//   to work around a bug in gcc 3.4.4 and 4.0.1.  If we had written
-//
-//     CompileAssert<bool(expr)>
-//
-//   instead, these compilers will refuse to compile
-//
-//     GTEST_COMPILE_ASSERT_(5 > 0, some_message);
-//
-//   (They seem to think the ">" in "5 > 0" marks the end of the
-//   template argument list.)
-//
-// - The array size is (bool(expr) ? 1 : -1), instead of simply
-//
-//     ((expr) ? 1 : -1).
-//
-//   This is to avoid running into a bug in MS VC 7.1, which
-//   causes ((0.0) ? 1 : -1) to incorrectly evaluate to 1.
-
-// StaticAssertTypeEqHelper is used by StaticAssertTypeEq defined in gtest.h.
-//
-// This template is declared, but intentionally undefined.
-template <typename T1, typename T2>
-struct StaticAssertTypeEqHelper;
-
-template <typename T>
-struct StaticAssertTypeEqHelper<T, T> {
-  enum { value = true };
-};
-
-// Evaluates to the number of elements in 'array'.
-#define GTEST_ARRAY_SIZE_(array) (sizeof(array) / sizeof(array[0]))
-
-#if GTEST_HAS_GLOBAL_STRING
-typedef ::string string;
-#else
-typedef ::std::string string;
-#endif  // GTEST_HAS_GLOBAL_STRING
-
-#if GTEST_HAS_GLOBAL_WSTRING
-typedef ::wstring wstring;
-#elif GTEST_HAS_STD_WSTRING
-typedef ::std::wstring wstring;
-#endif  // GTEST_HAS_GLOBAL_WSTRING
-
-// A helper for suppressing warnings on constant condition.  It just
-// returns 'condition'.
-GTEST_API_ bool IsTrue(bool condition);
-
-// Defines scoped_ptr.
-
-// This implementation of scoped_ptr is PARTIAL - it only contains
-// enough stuff to satisfy Google Test's need.
-template <typename T>
-class scoped_ptr {
- public:
-  typedef T element_type;
-
-  explicit scoped_ptr(T* p = NULL) : ptr_(p) {}
-  ~scoped_ptr() { reset(); }
-
-  T& operator*() const { return *ptr_; }
-  T* operator->() const { return ptr_; }
-  T* get() const { return ptr_; }
-
-  T* release() {
-    T* const ptr = ptr_;
-    ptr_ = NULL;
-    return ptr;
-  }
-
-  void reset(T* p = NULL) {
-    if (p != ptr_) {
-      if (IsTrue(sizeof(T) > 0)) {  // Makes sure T is a complete type.
-        delete ptr_;
-      }
-      ptr_ = p;
-    }
-  }
-
-  friend void swap(scoped_ptr& a, scoped_ptr& b) {
-    using std::swap;
-    swap(a.ptr_, b.ptr_);
-  }
-
- private:
-  T* ptr_;
-
-  GTEST_DISALLOW_COPY_AND_ASSIGN_(scoped_ptr);
-};
-
-// Defines RE.
-
-// A simple C++ wrapper for <regex.h>.  It uses the POSIX Extended
-// Regular Expression syntax.
-class GTEST_API_ RE {
- public:
-  // A copy constructor is required by the Standard to initialize object
-  // references from r-values.
-  RE(const RE& other) { Init(other.pattern()); }
-
-  // Constructs an RE from a string.
-  RE(const ::std::string& regex) { Init(regex.c_str()); }  // NOLINT
-
-#if GTEST_HAS_GLOBAL_STRING
-
-  RE(const ::string& regex) { Init(regex.c_str()); }  // NOLINT
-
-#endif  // GTEST_HAS_GLOBAL_STRING
-
-  RE(const char* regex) { Init(regex); }  // NOLINT
-  ~RE();
-
-  // Returns the string representation of the regex.
-  const char* pattern() const { return pattern_; }
-
-  // FullMatch(str, re) returns true iff regular expression re matches
-  // the entire str.
-  // PartialMatch(str, re) returns true iff regular expression re
-  // matches a substring of str (including str itself).
-  //
-  // TODO(wan@google.com): make FullMatch() and PartialMatch() work
-  // when str contains NUL characters.
-  static bool FullMatch(const ::std::string& str, const RE& re) {
-    return FullMatch(str.c_str(), re);
-  }
-  static bool PartialMatch(const ::std::string& str, const RE& re) {
-    return PartialMatch(str.c_str(), re);
-  }
-
-#if GTEST_HAS_GLOBAL_STRING
-
-  static bool FullMatch(const ::string& str, const RE& re) {
-    return FullMatch(str.c_str(), re);
-  }
-  static bool PartialMatch(const ::string& str, const RE& re) {
-    return PartialMatch(str.c_str(), re);
-  }
-
-#endif  // GTEST_HAS_GLOBAL_STRING
-
-  static bool FullMatch(const char* str, const RE& re);
-  static bool PartialMatch(const char* str, const RE& re);
-
- private:
-  void Init(const char* regex);
-
-  // We use a const char* instead of an std::string, as Google Test used to be
-  // used where std::string is not available.  TODO(wan@google.com): change to
-  // std::string.
-  const char* pattern_;
-  bool is_valid_;
-
-#if GTEST_USES_POSIX_RE
-
-  regex_t full_regex_;     // For FullMatch().
-  regex_t partial_regex_;  // For PartialMatch().
-
-#else  // GTEST_USES_SIMPLE_RE
-
-  const char* full_pattern_;  // For FullMatch();
-
-#endif
-
-  GTEST_DISALLOW_ASSIGN_(RE);
-};
-
-// Formats a source file path and a line number as they would appear
-// in an error message from the compiler used to compile this code.
-GTEST_API_ ::std::string FormatFileLocation(const char* file, int line);
-
-// Formats a file location for compiler-independent XML output.
-// Although this function is not platform dependent, we put it next to
-// FormatFileLocation in order to contrast the two functions.
-GTEST_API_ ::std::string FormatCompilerIndependentFileLocation(const char* file,
-                                                               int line);
-
-// Defines logging utilities:
-//   GTEST_LOG_(severity) - logs messages at the specified severity level. The
-//                          message itself is streamed into the macro.
-//   LogToStderr()  - directs all log messages to stderr.
-//   FlushInfoLog() - flushes informational log messages.
-
-enum GTestLogSeverity {
-  GTEST_INFO,
-  GTEST_WARNING,
-  GTEST_ERROR,
-  GTEST_FATAL
-};
-
-// Formats log entry severity, provides a stream object for streaming the
-// log message, and terminates the message with a newline when going out of
-// scope.
-class GTEST_API_ GTestLog {
- public:
-  GTestLog(GTestLogSeverity severity, const char* file, int line);
-
-  // Flushes the buffers and, if severity is GTEST_FATAL, aborts the program.
-  ~GTestLo

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/50] [abbrv] hadoop git commit: HADOOP-13257. Improve Azure Data Lake contract tests. Contributed by Vishwajeet Dusane

Posted by st...@apache.org.
HADOOP-13257. Improve Azure Data Lake contract tests. Contributed by Vishwajeet Dusane


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

Branch: refs/heads/HADOOP-13345
Commit: 4113ec5fa5ca049ebaba039b1faf3911c6a34f7b
Parents: 51211a7
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 2 15:54:57 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 2 15:54:57 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |  24 +-
 .../org/apache/hadoop/fs/adl/TestAdlRead.java   |   6 +-
 .../apache/hadoop/fs/adl/TestListStatus.java    |   6 +-
 .../fs/adl/live/TestAdlContractAppendLive.java  |  11 +-
 .../fs/adl/live/TestAdlContractConcatLive.java  |  23 +-
 .../fs/adl/live/TestAdlContractCreateLive.java  |  19 +-
 .../fs/adl/live/TestAdlContractDeleteLive.java  |  11 +-
 .../live/TestAdlContractGetFileStatusLive.java  |  36 ++
 .../fs/adl/live/TestAdlContractMkdirLive.java   |  25 +-
 .../fs/adl/live/TestAdlContractOpenLive.java    |  11 +-
 .../fs/adl/live/TestAdlContractRenameLive.java  |  30 +-
 .../fs/adl/live/TestAdlContractRootDirLive.java |  19 +-
 .../fs/adl/live/TestAdlContractSeekLive.java    |  11 +-
 .../live/TestAdlDifferentSizeWritesLive.java    |  69 ++--
 .../live/TestAdlFileContextCreateMkdirLive.java |  67 ++++
 .../TestAdlFileContextMainOperationsLive.java   |  99 ++++++
 .../adl/live/TestAdlFileSystemContractLive.java |  57 +---
 .../live/TestAdlInternalCreateNonRecursive.java | 134 ++++++++
 .../fs/adl/live/TestAdlPermissionLive.java      | 116 +++++++
 .../adl/live/TestAdlSupportedCharsetInPath.java | 334 +++++++++++++++++++
 .../apache/hadoop/fs/adl/live/TestMetadata.java | 111 ++++++
 21 files changed, 995 insertions(+), 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index 9083afc..bd43c52 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -346,7 +346,6 @@ public class AdlFileSystem extends FileSystem {
    * @see #setPermission(Path, FsPermission)
    * @deprecated API only for 0.20-append
    */
-  @Deprecated
   @Override
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flags, int bufferSize, short replication,
@@ -471,6 +470,10 @@ public class AdlFileSystem extends FileSystem {
   @Override
   public boolean rename(final Path src, final Path dst) throws IOException {
     statistics.incrementWriteOps(1);
+    if (toRelativeFilePath(src).equals("/")) {
+      return false;
+    }
+
     return adlClient.rename(toRelativeFilePath(src), toRelativeFilePath(dst));
   }
 
@@ -522,9 +525,24 @@ public class AdlFileSystem extends FileSystem {
   public boolean delete(final Path path, final boolean recursive)
       throws IOException {
     statistics.incrementWriteOps(1);
+    String relativePath = toRelativeFilePath(path);
+    // Delete on root directory not supported.
+    if (relativePath.equals("/")) {
+      // This is important check after recent commit
+      // HADOOP-12977 and HADOOP-13716 validates on root for
+      // 1. if root is empty and non recursive delete then return false.
+      // 2. if root is non empty and non recursive delete then throw exception.
+      if (!recursive
+          && adlClient.enumerateDirectory(toRelativeFilePath(path), 1).size()
+          > 0) {
+        throw new IOException("Delete on root is not supported.");
+      }
+      return false;
+    }
+
     return recursive ?
-        adlClient.deleteRecursive(toRelativeFilePath(path)) :
-        adlClient.delete(toRelativeFilePath(path));
+        adlClient.deleteRecursive(relativePath) :
+        adlClient.delete(relativePath);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
index 734256a..172663c 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAdlRead.java
@@ -102,7 +102,7 @@ public class TestAdlRead extends AdlMockWebServer {
       n += count;
     }
 
-    Assert.assertEquals(testData.getActualData().length, expectedData.length);
+    Assert.assertEquals(expectedData.length, testData.getActualData().length);
     Assert.assertArrayEquals(expectedData, testData.getActualData());
     in.close();
     if (testData.isCheckOfNoOfCalls()) {
@@ -119,8 +119,8 @@ public class TestAdlRead extends AdlMockWebServer {
     for (int i = 0; i < 1000; ++i) {
       int position = random.nextInt(testData.getActualData().length);
       in.seek(position);
-      Assert.assertEquals(in.getPos(), position);
-      Assert.assertEquals(in.read(), testData.getActualData()[position] & 0xFF);
+      Assert.assertEquals(position, in.getPos());
+      Assert.assertEquals(testData.getActualData()[position] & 0xFF, in.read());
     }
     in.close();
     if (testData.isCheckOfNoOfCalls()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
index dd27a10..c151e89 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
@@ -50,7 +50,7 @@ public class TestListStatus extends AdlMockWebServer {
         .listStatus(new Path("/test1/test2"));
     long endTime = Time.monotonicNow();
     LOG.debug("Time : " + (endTime - startTime));
-    Assert.assertEquals(ls.length, 10);
+    Assert.assertEquals(10, ls.length);
 
     getMockServer().enqueue(new MockResponse().setResponseCode(200)
         .setBody(TestADLResponseData.getListFileStatusJSONResponse(200)));
@@ -58,7 +58,7 @@ public class TestListStatus extends AdlMockWebServer {
     ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
     endTime = Time.monotonicNow();
     LOG.debug("Time : " + (endTime - startTime));
-    Assert.assertEquals(ls.length, 200);
+    Assert.assertEquals(200, ls.length);
 
     getMockServer().enqueue(new MockResponse().setResponseCode(200)
         .setBody(TestADLResponseData.getListFileStatusJSONResponse(2048)));
@@ -66,7 +66,7 @@ public class TestListStatus extends AdlMockWebServer {
     ls = getMockAdlFileSystem().listStatus(new Path("/test1/test2"));
     endTime = Time.monotonicNow();
     LOG.debug("Time : " + (endTime - startTime));
-    Assert.assertEquals(ls.length, 2048);
+    Assert.assertEquals(2048, ls.length);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
index 83390af..ffe6dd3 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractAppendLive.java
@@ -23,11 +23,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
 import org.junit.Test;
 
 /**
- * Verify Adls APPEND semantics compliance with Hadoop.
+ * Test Append on Adl file system.
  */
 public class TestAdlContractAppendLive extends AbstractContractAppendTest {
 
@@ -42,12 +41,4 @@ public class TestAdlContractAppendLive extends AbstractContractAppendTest {
     ContractTestUtils.unsupported("Skipping since renaming file in append "
         + "mode not supported in Adl");
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
index 8474e9c..60d30ac 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractConcatLive.java
@@ -20,14 +20,15 @@
 package org.apache.hadoop.fs.adl.live;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
 /**
- * Verify Adls CONCAT semantics compliance with Hadoop.
+ * Test concat on Adl file system.
  */
 public class TestAdlContractConcatLive extends AbstractContractConcatTest {
 
@@ -36,17 +37,13 @@ public class TestAdlContractConcatLive extends AbstractContractConcatTest {
     return new AdlStorageContract(configuration);
   }
 
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
   @Test
   public void testConcatMissingTarget() throws Throwable {
-    ContractTestUtils.unsupported("BUG : Adl to support expectation from "
-        + "concat on missing targets.");
+    Path testPath = path("test");
+    Path zeroByteFile = new Path(testPath, "zero.txt");
+    Path target = new Path(testPath, "target");
+    touch(getFileSystem(), zeroByteFile);
+    // Concat on missing target is allowed on Adls file system.
+    getFileSystem().concat(target, new Path[] {zeroByteFile});
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
index 907c50c..06347e9 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractCreateLive.java
@@ -22,12 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls CREATE semantics compliance with Hadoop.
+ * Test creating files, overwrite options.
  */
 public class TestAdlContractCreateLive extends AbstractContractCreateTest {
 
@@ -35,18 +32,4 @@ public class TestAdlContractCreateLive extends AbstractContractCreateTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testOverwriteEmptyDirectory() throws Throwable {
-    ContractTestUtils
-        .unsupported("BUG : Adl to support override empty " + "directory.");
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
index 30eaec7..6961f15 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractDeleteLive.java
@@ -22,10 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.junit.Before;
 
 /**
- * Verify Adls DELETE semantics compliance with Hadoop.
+ * Test delete contract test.
  */
 public class TestAdlContractDeleteLive extends AbstractContractDeleteTest {
 
@@ -33,12 +32,4 @@ public class TestAdlContractDeleteLive extends AbstractContractDeleteTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java
new file mode 100644
index 0000000..d50dd68
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractGetFileStatusLive.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test getFileStatus contract test.
+ */
+public class TestAdlContractGetFileStatusLive extends
+    AbstractContractGetFileStatusTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration configuration) {
+    return new AdlStorageContract(configuration);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
index e498110..5e760c5 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractMkdirLive.java
@@ -22,34 +22,13 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls MKDIR semantics compliance with Hadoop.
+ * Test Mkdir contract on Adl storage file system.
  */
 public class TestAdlContractMkdirLive extends AbstractContractMkdirTest {
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new AdlStorageContract(conf);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testMkdirOverParentFile() throws Throwable {
-    ContractTestUtils.unsupported("Not supported by Adl");
-  }
-
-  @Test
-  public void testNoMkdirOverFile() throws Throwable {
-    ContractTestUtils.unsupported("Not supported by Adl");
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
index 2bb2095..7a35d2c 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractOpenLive.java
@@ -22,10 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.junit.Before;
 
 /**
- * Verify Adls OPEN/READ semantics compliance with Hadoop.
+ * Test OPEN - read API.
  */
 public class TestAdlContractOpenLive extends AbstractContractOpenTest {
 
@@ -33,12 +32,4 @@ public class TestAdlContractOpenLive extends AbstractContractOpenTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
index 06063c5..d72d35e 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRenameLive.java
@@ -22,12 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls RENAME semantics compliance with Hadoop.
+ * Test rename contract test cases on Adl file system.
  */
 public class TestAdlContractRenameLive extends AbstractContractRenameTest {
 
@@ -35,29 +32,4 @@ public class TestAdlContractRenameLive extends AbstractContractRenameTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testRenameFileOverExistingFile() throws Throwable {
-    ContractTestUtils
-        .unsupported("BUG : Adl to support full complete POSIX" + "behaviour");
-  }
-
-  @Test
-  public void testRenameFileNonexistentDir() throws Throwable {
-    ContractTestUtils
-        .unsupported("BUG : Adl to support create dir is not " + "exist");
-  }
-
-  @Test
-  public void testRenameWithNonEmptySubDir() throws Throwable {
-    ContractTestUtils.unsupported("BUG : Adl to support non empty dir move.");
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
index bf4e549..8ebc632 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractRootDirLive.java
@@ -22,12 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Verify Adls root level operation support.
+ * Test operation on root level.
  */
 public class TestAdlContractRootDirLive
     extends AbstractContractRootDirectoryTest {
@@ -35,18 +32,4 @@ public class TestAdlContractRootDirLive
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
-
-  @Test
-  public void testRmNonEmptyRootDirNonRecursive() throws Throwable {
-    ContractTestUtils.unsupported(
-        "BUG : Adl should throw exception instred " + "of returning false.");
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
index 0976464..62423b6 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlContractSeekLive.java
@@ -22,10 +22,9 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.junit.Before;
 
 /**
- * Verify Adls OPEN/READ seek operation support.
+ * Test seek operation on Adl file system.
  */
 public class TestAdlContractSeekLive extends AbstractContractSeekTest {
 
@@ -33,12 +32,4 @@ public class TestAdlContractSeekLive extends AbstractContractSeekTest {
   protected AbstractFSContract createContract(Configuration configuration) {
     return new AdlStorageContract(configuration);
   }
-
-  @Before
-  @Override
-  public void setup() throws Exception {
-    org.junit.Assume
-        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
-    super.setup();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
index 8f53400..5421e0b 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlDifferentSizeWritesLive.java
@@ -23,27 +23,63 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Random;
+import java.util.UUID;
+
+import static org.apache.hadoop.fs.adl.AdlConfKeys.WRITE_BUFFER_SIZE_KEY;
 
 /**
- * Verify different data segment size writes ensure the integrity and
- * order of the data.
+ * Verify data integrity with different data sizes with buffer size.
  */
+@RunWith(Parallelized.class)
 public class TestAdlDifferentSizeWritesLive {
+  private static Random rand = new Random();
+  private int totalSize;
+  private int chunkSize;
+
+  public TestAdlDifferentSizeWritesLive(int totalSize, int chunkSize) {
+    this.totalSize = totalSize;
+    this.chunkSize = chunkSize;
+  }
 
   public static byte[] getRandomByteArrayData(int size) {
     byte[] b = new byte[size];
-    Random rand = new Random();
     rand.nextBytes(b);
     return b;
   }
 
+  @Parameterized.Parameters(name = "{index}: Data Size [{0}] ; Chunk Size "
+      + "[{1}]")
+  public static Collection testDataForIntegrityTest() {
+    return Arrays.asList(
+        new Object[][] {{4 * 1024, 1 * 1024}, {4 * 1024, 7 * 1024},
+            {4 * 1024, 10}, {2 * 1024, 10}, {1 * 1024, 10}, {100, 1},
+            {4 * 1024, 1 * 1024}, {7 * 1024, 2 * 1024}, {9 * 1024, 2 * 1024},
+            {10 * 1024, 3 * 1024}, {10 * 1024, 1 * 1024},
+            {10 * 1024, 8 * 1024}});
+  }
+
+  @BeforeClass
+  public static void cleanUpParent() throws IOException, URISyntaxException {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      Path path = new Path("/test/dataIntegrityCheck/");
+      FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+      fs.delete(path, true);
+    }
+  }
+
   @Before
   public void setup() throws Exception {
     org.junit.Assume
@@ -51,32 +87,17 @@ public class TestAdlDifferentSizeWritesLive {
   }
 
   @Test
-  public void testSmallDataWrites() throws IOException {
-    testDataIntegrity(4 * 1024 * 1024, 1 * 1024);
-    testDataIntegrity(4 * 1024 * 1024, 7 * 1024);
-    testDataIntegrity(4 * 1024 * 1024, 10);
-    testDataIntegrity(2 * 1024 * 1024, 10);
-    testDataIntegrity(1 * 1024 * 1024, 10);
-    testDataIntegrity(100, 1);
-  }
-
-  @Test
-  public void testMediumDataWrites() throws IOException {
-    testDataIntegrity(4 * 1024 * 1024, 1 * 1024 * 1024);
-    testDataIntegrity(7 * 1024 * 1024, 2 * 1024 * 1024);
-    testDataIntegrity(9 * 1024 * 1024, 2 * 1024 * 1024);
-    testDataIntegrity(10 * 1024 * 1024, 3 * 1024 * 1024);
-  }
-
-  private void testDataIntegrity(int totalSize, int chunkSize)
-      throws IOException {
-    Path path = new Path("/test/dataIntegrityCheck");
+  public void testDataIntegrity() throws IOException {
+    Path path = new Path(
+        "/test/dataIntegrityCheck/" + UUID.randomUUID().toString());
     FileSystem fs = null;
+    AdlStorageConfiguration.getConfiguration()
+        .setInt(WRITE_BUFFER_SIZE_KEY, 4 * 1024);
     try {
       fs = AdlStorageConfiguration.createStorageConnector();
     } catch (URISyntaxException e) {
       throw new IllegalStateException("Can not initialize ADL FileSystem. "
-          + "Please check fs.defaultFS property.", e);
+          + "Please check test.fs.adl.name property.", e);
     }
     byte[] expectedData = getRandomByteArrayData(totalSize);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.java
new file mode 100644
index 0000000..5166de1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextCreateMkdirLive.java
@@ -0,0 +1,67 @@
+/*
+ * 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.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import java.net.URI;
+import java.util.UUID;
+
+/**
+ * Test file context Create/Mkdir operation.
+ */
+public class TestAdlFileContextCreateMkdirLive
+    extends FileContextCreateMkdirBaseTest {
+  private static final String KEY_FILE_SYSTEM = "test.fs.adl.name";
+
+  @BeforeClass
+  public static void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = AdlStorageConfiguration.getConfiguration();
+    String fileSystem = conf.get(KEY_FILE_SYSTEM);
+    if (fileSystem == null || fileSystem.trim().length() == 0) {
+      throw new Exception("Default file system not configured.");
+    }
+    URI uri = new URI(fileSystem);
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+    fc = FileContext.getFileContext(
+        new DelegateToFileSystem(uri, fs, conf, fs.getScheme(), false) {
+        }, conf);
+    super.setUp();
+  }
+
+  @Override
+  protected FileContextTestHelper createFileContextHelper() {
+    // On Windows, root directory path is created from local running directory.
+    // Adl does not support ':' as part of the path which results in failure.
+    return new FileContextTestHelper(UUID.randomUUID().toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java
new file mode 100644
index 0000000..ee10da7
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileContextMainOperationsLive.java
@@ -0,0 +1,99 @@
+/*
+ * 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.adl.live;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.UUID;
+
+import static org.apache.hadoop.util.Shell.WINDOWS;
+
+/**
+ * Run collection of tests for the {@link FileContext}.
+ */
+public class TestAdlFileContextMainOperationsLive
+    extends FileContextMainOperationsBaseTest {
+
+  private static final String KEY_FILE_SYSTEM = "test.fs.adl.name";
+
+  @BeforeClass
+  public static void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = AdlStorageConfiguration.getConfiguration();
+    String fileSystem = conf.get(KEY_FILE_SYSTEM);
+    if (fileSystem == null || fileSystem.trim().length() == 0) {
+      throw new Exception("Default file system not configured.");
+    }
+    URI uri = new URI(fileSystem);
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+    fc = FileContext.getFileContext(
+        new DelegateToFileSystem(uri, fs, conf, fs.getScheme(), false) {
+        }, conf);
+    super.setUp();
+  }
+
+  @Override
+  protected FileContextTestHelper createFileContextHelper() {
+    // On Windows, root directory path is created from local running directory.
+    // Adl does not support ':' as part of the path which results in failure.
+    //    return new FileContextTestHelper(GenericTestUtils
+    // .getRandomizedTestDir()
+    //        .getAbsolutePath().replaceAll(":",""));
+    return new FileContextTestHelper(UUID.randomUUID().toString());
+  }
+
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return false;
+  }
+
+  @Override
+  public void testWorkingDirectory() throws Exception {
+    if (WINDOWS) {
+      // TODO :Fix is required in Hadoop shell to support windows permission
+      // set.
+      // The test is failing with NPE on windows platform only, with Linux
+      // platform test passes.
+      Assume.assumeTrue(false);
+    } else {
+      super.testWorkingDirectory();
+    }
+  }
+
+  @Override
+  public void testUnsupportedSymlink() throws IOException {
+    Assume.assumeTrue(false);
+  }
+
+  @Test
+  public void testSetVerifyChecksum() throws IOException {
+    Assume.assumeTrue(false);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
index 0df7d05..657947e 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlFileSystemContractLive.java
@@ -22,12 +22,13 @@ package org.apache.hadoop.fs.adl.live;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
+import org.junit.Assume;
+import org.junit.Before;
 
 import java.io.IOException;
 
 /**
- * Verify Adls adhere to Hadoop file system semantics.
+ * Test Base contract tests on Adl file system.
  */
 public class TestAdlFileSystemContractLive extends FileSystemContractBaseTest {
   private FileSystem adlStore;
@@ -60,52 +61,8 @@ public class TestAdlFileSystemContractLive extends FileSystemContractBaseTest {
     }
   }
 
-  public void testGetFileStatus() throws IOException {
-    if (!AdlStorageConfiguration.isContractTestEnabled()) {
-      return;
-    }
-
-    Path testPath = new Path("/test/adltest");
-    if (adlStore.exists(testPath)) {
-      adlStore.delete(testPath, false);
-    }
-
-    adlStore.create(testPath).close();
-    assertTrue(adlStore.delete(testPath, false));
-  }
-
-  /**
-   * The following tests are failing on Azure Data Lake and the Azure Data Lake
-   * file system code needs to be modified to make them pass.
-   * A separate work item has been opened for this.
-   */
-  @Test
-  @Override
-  public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception {
-    // BUG : Adl should return exception instead of false.
-  }
-
-  @Test
-  @Override
-  public void testMkdirsWithUmask() throws Exception {
-    // Support under implementation in Adl
-  }
-
-  @Test
-  @Override
-  public void testMoveFileUnderParent() throws Exception {
-    // BUG: Adl server should return expected status code.
-  }
-
-  @Test
-  @Override
-  public void testRenameFileToSelf() throws Exception {
-    // BUG: Adl server should return expected status code.
-  }
-
-  @Test
-  @Override
-  public void testRenameToDirWithSamePrefixAllowed() throws Exception {
-    // BUG: Adl server should return expected status code.
+  @Before
+  public void skipTestCheck() {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java
new file mode 100644
index 0000000..7e11a54
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlInternalCreateNonRecursive.java
@@ -0,0 +1,134 @@
+/*
+ * 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.adl.live;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+
+/**
+ * Test createNonRecursive API.
+ */
+@RunWith(Parallelized.class)
+public class TestAdlInternalCreateNonRecursive {
+  private Path inputFileName;
+  private FsPermission inputPermission;
+  private boolean inputOverride;
+  private boolean inputFileAlreadyExist;
+  private boolean inputParentAlreadyExist;
+  private Class<IOException> expectedExceptionType;
+  private FileSystem adlStore;
+
+  public TestAdlInternalCreateNonRecursive(String testScenario, String fileName,
+      FsPermission permission, boolean override, boolean fileAlreadyExist,
+      boolean parentAlreadyExist, Class<IOException> exceptionType) {
+
+    // Random parent path for each test so that parallel execution does not fail
+    // other running test.
+    inputFileName = new Path(
+        "/test/createNonRecursive/" + UUID.randomUUID().toString(), fileName);
+    inputPermission = permission;
+    inputFileAlreadyExist = fileAlreadyExist;
+    inputOverride = override;
+    inputParentAlreadyExist = parentAlreadyExist;
+    expectedExceptionType = exceptionType;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection adlCreateNonRecursiveTestData()
+      throws UnsupportedEncodingException {
+    /*
+      Test Data
+      File name, Permission, Override flag, File already exist, Parent
+      already exist
+      shouldCreateSucceed, expectedExceptionIfFileCreateFails
+
+      File already exist and Parent already exist are mutually exclusive.
+    */
+    return Arrays.asList(new Object[][] {
+        {"CNR - When file do not exist.", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), false, false, true, null},
+        {"CNR - When file exist. Override false", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), false, true, true,
+            FileAlreadyExistsException.class},
+        {"CNR - When file exist. Override true", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), true, true, true, null},
+
+        //TODO: This test is skipped till the fixes are not made it to prod.
+        /*{ "CNR - When parent do no exist.", UUID.randomUUID().toString(),
+            FsPermission.getFileDefault(), false, false, true, false,
+            IOException.class }*/});
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    adlStore = AdlStorageConfiguration.createStorageConnector();
+  }
+
+  @Test
+  public void testCreateNonRecursiveFunctionality() throws IOException {
+    if (inputFileAlreadyExist) {
+      FileSystem.create(adlStore, inputFileName, inputPermission);
+    }
+
+    // Mutually exclusive to inputFileAlreadyExist
+    if (inputParentAlreadyExist) {
+      adlStore.mkdirs(inputFileName.getParent());
+    } else {
+      adlStore.delete(inputFileName.getParent(), true);
+    }
+
+    try {
+      adlStore.createNonRecursive(inputFileName, inputPermission, inputOverride,
+          CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT,
+          adlStore.getDefaultReplication(inputFileName),
+          adlStore.getDefaultBlockSize(inputFileName), null);
+    } catch (IOException e) {
+
+      if (expectedExceptionType == null) {
+        throw e;
+      }
+
+      Assert.assertEquals(expectedExceptionType, e.getClass());
+      return;
+    }
+
+    if (expectedExceptionType != null) {
+      Assert.fail("CreateNonRecursive should have failed with exception "
+          + expectedExceptionType.getName());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java
new file mode 100644
index 0000000..dd7c10d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlPermissionLive.java
@@ -0,0 +1,116 @@
+/*
+ * 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.adl.live;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.*;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+
+/**
+ * Test ACL permission on file/folder on Adl file system.
+ */
+@RunWith(Parallelized.class)
+public class TestAdlPermissionLive {
+
+  private static Path testRoot = new Path("/test");
+  private FsPermission permission;
+  private Path path;
+  private FileSystem adlStore;
+
+  public TestAdlPermissionLive(FsPermission testPermission) {
+    permission = testPermission;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection adlCreateNonRecursiveTestData()
+      throws UnsupportedEncodingException {
+    /*
+      Test Data
+      File/Folder name, User permission, Group permission, Other Permission,
+      Parent already exist
+      shouldCreateSucceed, expectedExceptionIfFileCreateFails
+    */
+    final Collection<Object[]> datas = new ArrayList<>();
+    for (FsAction g : FsAction.values()) {
+      for (FsAction o : FsAction.values()) {
+        datas.add(new Object[] {new FsPermission(FsAction.ALL, g, o)});
+      }
+    }
+    return datas;
+  }
+
+  @AfterClass
+  public static void cleanUp() throws IOException, URISyntaxException {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      Assert.assertTrue(AdlStorageConfiguration.createStorageConnector()
+          .delete(testRoot, true));
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    adlStore = AdlStorageConfiguration.createStorageConnector();
+  }
+
+  @Test
+  public void testFilePermission() throws IOException {
+    path = new Path(testRoot, UUID.randomUUID().toString());
+    adlStore.getConf()
+        .set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000");
+
+    adlStore.mkdirs(path.getParent(),
+        new FsPermission(FsAction.ALL, FsAction.WRITE, FsAction.NONE));
+    adlStore.removeDefaultAcl(path.getParent());
+
+    adlStore.create(path, permission, true, 1024, (short) 1, 1023, null);
+    FileStatus status = adlStore.getFileStatus(path);
+    Assert.assertEquals(permission, status.getPermission());
+  }
+
+  @Test
+  public void testFolderPermission() throws IOException {
+    path = new Path(testRoot, UUID.randomUUID().toString());
+    adlStore.getConf()
+        .set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000");
+    adlStore.mkdirs(path.getParent(),
+        new FsPermission(FsAction.ALL, FsAction.WRITE, FsAction.NONE));
+    adlStore.removeDefaultAcl(path.getParent());
+
+    adlStore.mkdirs(path, permission);
+    FileStatus status = adlStore.getFileStatus(path);
+    Assert.assertEquals(permission, status.getPermission());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java
new file mode 100644
index 0000000..d80b6bf
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSupportedCharsetInPath.java
@@ -0,0 +1,334 @@
+/*
+ * 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.adl.live;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.common.Parallelized;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URISyntaxException;
+import java.util.*;
+
+/**
+ * Test supported ASCII, UTF-8 character set supported by Adl storage file
+ * system on file/folder operation.
+ */
+@RunWith(Parallelized.class)
+public class TestAdlSupportedCharsetInPath {
+
+  private static final String TEST_ROOT = "/test/";
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestAdlSupportedCharsetInPath.class);
+  private String path;
+
+  public TestAdlSupportedCharsetInPath(String filePath) {
+    path = filePath;
+  }
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> adlCharTestData()
+      throws UnsupportedEncodingException {
+
+    ArrayList<String> filePathList = new ArrayList<>();
+    for (int i = 32; i < 127; ++i) {
+      String specialChar = (char) i + "";
+      if (i >= 48 && i <= 57) {
+        continue;
+      }
+
+      if (i >= 65 && i <= 90) {
+        continue;
+      }
+
+      if (i >= 97 && i <= 122) {
+        continue;
+      }
+
+      // Special char at start of the path
+      if (i != 92 && i != 58 && i != 46 && i != 47) {
+        filePathList.add(specialChar + "");
+      }
+
+      // Special char at end of string
+      if (i != 92 && i != 47 && i != 58) {
+        filePathList.add("file " + i + " " + specialChar);
+      }
+
+      // Special char in between string
+      if (i != 47 && i != 58 && i != 92) {
+        filePathList.add("file " + i + " " + specialChar + "_name");
+      }
+    }
+
+    filePathList.add("a  ");
+    filePathList.add("a..b");
+    fillUnicodes(filePathList);
+    Collection<Object[]> result = new ArrayList<>();
+    for (String item : filePathList) {
+      result.add(new Object[] {item});
+    }
+    return result;
+  }
+
+  private static void fillUnicodes(ArrayList<String> filePathList) {
+    // Unicode characters
+    filePathList.add("\u0627\u0644\u0628\u064a\u0627\u0646\u0627\u062a \u0627\u0644\u0643\u0628\u064a\u0631\u0629"); // Arabic
+    filePathList.add("T� dh�nat i madh"); // Albanian
+    filePathList.add("\u0574\u0565\u056e \u057f\u057e\u0575\u0561\u056c\u0576\u0565\u0580\u0568"); // Armenian
+    filePathList.add("b�y�k data"); // Azerbaijani
+    filePathList.add("\u0432\u044f\u043b\u0456\u043a\u0456\u044f \u0434\u0430\u0434\u0437\u0435\u043d\u044b\u044f"); // Belarusian,
+    filePathList.add("\u09ac\u09bf\u0997 \u09a1\u09c7\u099f\u09be"); // Bengali
+    filePathList.add("veliki podataka"); // Bosnian
+    filePathList.add("\u0433\u043e\u043b\u044f\u043c\u0430 \u0434\u0430\u043d\u043d\u0438"); // Bulgarian
+    filePathList.add("\u5927\u6570\u636e"); // Chinese - Simplified
+    filePathList.add("\u5927\u6578\u64da"); // Chinese - Traditional
+    filePathList.add("\u10d3\u10d8\u10d3\u10d8 \u10db\u10dd\u10dc\u10d0\u10ea\u10d4\u10db\u10d7\u10d0"); // Georgian,
+    filePathList.add("gro�e Daten"); // German
+    filePathList.add("\u03bc\u03b5\u03b3\u03ac\u03bb\u03bf \u03b4\u03b5\u03b4\u03bf\u03bc\u03ad\u03bd\u03b1"); // Greek
+    filePathList.add("\u0aae\u0acb\u0a9f\u0abe \u0aae\u0abe\u0ab9\u0abf\u0aa4\u0ac0"); // Gujarati
+    filePathList.add("\u05e0\u05ea\u05d5\u05e0\u05d9\u05dd \u05d2\u05d3\u05d5\u05dc\u05d9\u05dd"); // Hebrew
+    filePathList.add("\u092c\u0921\u093c\u093e \u0921\u0947\u091f\u093e"); // Hindi
+    filePathList.add("st�r g�gn"); // Icelandic
+    filePathList.add("sonra� m�r"); // Irish
+    filePathList.add("\u30d3\u30c3\u30b0\u30c7\u30fc\u30bf"); // Japanese
+    filePathList.add("\u04af\u043b\u043a\u0435\u043d \u0434\u0435\u0440\u0435\u043a\u0442\u0435\u0440"); // Kazakh
+    filePathList.add("\u1791\u17b7\u1793\u17d2\u1793\u1793\u17d0\u1799\u1792\u17c6"); // Khmer
+    filePathList.add("\ube45 \ub370\uc774\ud130"); // Korean
+    filePathList.add("\u0e82\u0ecd\u0ec9\u0ea1\u0eb9\u0e99 \u0e82\u0eb0\u0eab\u0e99\u0eb2\u0e94\u0ec3\u0eab\u0e8d\u0ec8"); // Lao
+    filePathList.add("\u0433\u043e\u043b\u0435\u043c\u0438 \u043f\u043e\u0434\u0430\u0442\u043e\u0446\u0438"); // Macedonian
+    filePathList.add("\u0920\u0942\u0932\u094b \u0921\u093e\u091f\u093e"); // Nepali
+    filePathList.add("\u0d35\u0d32\u0d3f\u0d2f \u0d21\u0d3e\u0d31\u0d4d\u0d31"); // Malayalam
+    filePathList.add("\u092e\u094b\u0920\u0947 \u0921\u0947\u091f\u093e"); // Marathi
+    filePathList.add("\u0442\u043e\u043c \u043c\u044d\u0434\u044d\u044d\u043b\u044d\u043b"); // Mangolian
+    filePathList.add("\u0627\u0637\u0644\u0627\u0639\u0627\u062a \u0628\u0632\u0631\u06af"); // Persian
+    filePathList.add("\u0a35\u0a71\u0a21\u0a47 \u0a21\u0a3e\u0a1f\u0a47 \u0a28\u0a42\u0a70"); // Punjabi
+    filePathList.add("\u0431\u043e\u043b\u044c\u0448\u0438\u0435 \u0434\u0430\u043d\u043d\u044b\u0435"); // Russian
+    filePathList.add("\u0412\u0435\u043b\u0438\u043a\u0438 \u043f\u043e\u0434\u0430\u0442\u0430\u043a\u0430"); // Serbian
+    filePathList.add("\u0dc0\u0dd2\u0dc1\u0dcf\u0dbd \u0daf\u0dad\u0dca\u0dad"); // Sinhala
+    filePathList.add("big d�t"); // Slovak
+    filePathList.add("\u043c\u0430\u044a\u043b\u0443\u043c\u043e\u0442\u0438 \u043a\u0430\u043b\u043e\u043d"); // Tajik
+    filePathList.add("\u0baa\u0bc6\u0bb0\u0bbf\u0baf \u0ba4\u0bb0\u0bb5\u0bc1"); // Tamil
+    filePathList.add("\u0c2a\u0c46\u0c26\u0c4d\u0c26 \u0c21\u0c47\u0c1f\u0c3e"); // Telugu
+    filePathList.add("\u0e02\u0e49\u0e2d\u0e21\u0e39\u0e25\u0e43\u0e2b\u0e0d\u0e48"); // Thai
+    filePathList.add("b�y�k veri"); // Turkish
+    filePathList.add("\u0432\u0435\u043b\u0438\u043a\u0456 \u0434\u0430\u043d\u0456"); // Ukranian
+    filePathList.add("\u0628\u0691\u06d2 \u0627\u0639\u062f\u0627\u062f \u0648 \u0634\u0645\u0627\u0631"); // Urdu
+    filePathList.add("katta ma'lumotlar"); // Uzbek
+    filePathList.add("d\u1eef li\u1ec7u l\u1edbn"); // Vietanamese
+    filePathList.add("\u05d2\u05e8\u05d5\u05d9\u05e1 \u05d3\u05d0\u05b7\u05d8\u05df"); // Yiddish
+    filePathList.add("big idatha"); // Zulu
+    filePathList.add("rachel\u03c7");
+    filePathList.add("jessica\u03bf");
+    filePathList.add("sarah\u03b4");
+    filePathList.add("katie\u03bd");
+    filePathList.add("wendy\u03be");
+    filePathList.add("david\u03bc");
+    filePathList.add("priscilla\u03c5");
+    filePathList.add("oscar\u03b8");
+    filePathList.add("xavier\u03c7");
+    filePathList.add("gabriella\u03b8");
+    filePathList.add("david\u03c5");
+    filePathList.add("irene\u03bc");
+    filePathList.add("fred\u03c1");
+    filePathList.add("david\u03c4");
+    filePathList.add("ulysses\u03bd");
+    filePathList.add("gabriella\u03bc");
+    filePathList.add("zach\u03b6");
+    filePathList.add("gabriella\u03bb");
+    filePathList.add("ulysses\u03c6");
+    filePathList.add("david\u03c7");
+    filePathList.add("sarah\u03c3");
+    filePathList.add("holly\u03c8");
+    filePathList.add("nick\u03b1");
+    filePathList.add("ulysses\u03b9");
+    filePathList.add("mike\u03b2");
+    filePathList.add("priscilla\u03ba");
+    filePathList.add("wendy\u03b8");
+    filePathList.add("jessica\u03c2");
+    filePathList.add("fred\u03c7");
+    filePathList.add("fred\u03b6");
+    filePathList.add("sarah\u03ba");
+    filePathList.add("calvin\u03b7");
+    filePathList.add("xavier\u03c7");
+    filePathList.add("yuri\u03c7");
+    filePathList.add("ethan\u03bb");
+    filePathList.add("holly\u03b5");
+    filePathList.add("xavier\u03c3");
+    filePathList.add("victor\u03c4");
+    filePathList.add("wendy\u03b2");
+    filePathList.add("jessica\u03c2");
+    filePathList.add("quinn\u03c6");
+    filePathList.add("xavier\u03c5");
+    filePathList.add("nick\u03b9");
+    filePathList.add("rachel\u03c6");
+    filePathList.add("oscar\u03be");
+    filePathList.add("zach\u03b4");
+    filePathList.add("zach\u03bb");
+    filePathList.add("rachel\u03b1");
+    filePathList.add("jessica\u03c6");
+    filePathList.add("luke\u03c6");
+    filePathList.add("tom\u03b6");
+    filePathList.add("nick\u03be");
+    filePathList.add("nick\u03ba");
+    filePathList.add("ethan\u03b4");
+    filePathList.add("fred\u03c7");
+    filePathList.add("priscilla\u03b8");
+    filePathList.add("zach\u03be");
+    filePathList.add("xavier\u03be");
+    filePathList.add("zach\u03c8");
+    filePathList.add("ethan\u03b1");
+    filePathList.add("oscar\u03b9");
+    filePathList.add("irene\u03b4");
+    filePathList.add("irene\u03b6");
+    filePathList.add("victor\u03bf");
+    filePathList.add("wendy\u03b2");
+    filePathList.add("mike\u03c3");
+    filePathList.add("fred\u03bf");
+    filePathList.add("mike\u03b7");
+    filePathList.add("sarah\u03c1");
+    filePathList.add("quinn\u03b2");
+    filePathList.add("mike\u03c5");
+    filePathList.add("nick\u03b6");
+    filePathList.add("nick\u03bf");
+    filePathList.add("tom\u03ba");
+    filePathList.add("bob\u03bb");
+    filePathList.add("yuri\u03c0");
+    filePathList.add("david\u03c4");
+    filePathList.add("quinn\u03c0");
+    filePathList.add("mike\u03bb");
+    filePathList.add("david\u03b7");
+    filePathList.add("ethan\u03c4");
+    filePathList.add("nick\u03c6");
+    filePathList.add("yuri\u03bf");
+    filePathList.add("ethan\u03c5");
+    filePathList.add("bob\u03b8");
+    filePathList.add("david\u03bb");
+    filePathList.add("priscilla\u03be");
+    filePathList.add("nick\u03b3");
+    filePathList.add("luke\u03c5");
+    filePathList.add("irene\u03bb");
+    filePathList.add("xavier\u03bf");
+    filePathList.add("fred\u03c5");
+    filePathList.add("ulysses\u03bc");
+    filePathList.add("wendy\u03b3");
+    filePathList.add("zach\u03bb");
+    filePathList.add("rachel\u03c2");
+    filePathList.add("sarah\u03c0");
+    filePathList.add("alice\u03c8");
+    filePathList.add("bob\u03c4");
+  }
+
+  @AfterClass
+  public static void testReport() throws IOException, URISyntaxException {
+    if (!AdlStorageConfiguration.isContractTestEnabled()) {
+      return;
+    }
+
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+    fs.delete(new Path(TEST_ROOT), true);
+  }
+
+  @Test
+  public void testAllowedSpecialCharactersMkdir()
+      throws IOException, URISyntaxException {
+    Path parentPath = new Path(TEST_ROOT, UUID.randomUUID().toString() + "/");
+    Path specialFile = new Path(parentPath, path);
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+
+    Assert.assertTrue("Mkdir failed : " + specialFile, fs.mkdirs(specialFile));
+    Assert.assertTrue("File not Found after Mkdir success" + specialFile,
+        fs.exists(specialFile));
+    Assert.assertTrue("Not listed under parent " + parentPath,
+        contains(fs.listStatus(parentPath),
+            fs.makeQualified(specialFile).toString()));
+    Assert.assertTrue("Delete failed : " + specialFile,
+            fs.delete(specialFile, true));
+    Assert.assertFalse("File still exist after delete " + specialFile,
+        fs.exists(specialFile));
+  }
+
+  private boolean contains(FileStatus[] statuses, String remotePath) {
+    for (FileStatus status : statuses) {
+      if (status.getPath().toString().equals(remotePath)) {
+        return true;
+      }
+    }
+
+    Arrays.stream(statuses).forEach(s -> LOG.info(s.getPath().toString()));
+    return false;
+  }
+
+  @Before
+  public void setup() throws Exception {
+    org.junit.Assume
+        .assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+  }
+
+  @Test
+  public void testAllowedSpecialCharactersRename()
+      throws IOException, URISyntaxException {
+
+    String parentPath = TEST_ROOT + UUID.randomUUID().toString() + "/";
+    Path specialFile = new Path(parentPath + path);
+    Path anotherLocation = new Path(parentPath + UUID.randomUUID().toString());
+    FileSystem fs = AdlStorageConfiguration.createStorageConnector();
+
+    Assert.assertTrue("Could not create " + specialFile.toString(),
+        fs.createNewFile(specialFile));
+    Assert.assertTrue(
+        "Failed to rename " + specialFile.toString() + " --> " + anotherLocation
+            .toString(), fs.rename(specialFile, anotherLocation));
+    Assert.assertFalse("File should not be present after successful rename : "
+        + specialFile.toString(), fs.exists(specialFile));
+    Assert.assertTrue("File should be present after successful rename : "
+        + anotherLocation.toString(), fs.exists(anotherLocation));
+    Assert.assertFalse(
+        "Listed under parent whereas expected not listed : " + parentPath,
+        contains(fs.listStatus(new Path(parentPath)),
+            fs.makeQualified(specialFile).toString()));
+
+    Assert.assertTrue(
+        "Failed to rename " + anotherLocation.toString() + " --> " + specialFile
+            .toString(), fs.rename(anotherLocation, specialFile));
+    Assert.assertTrue(
+        "File should be present after successful rename : " + "" + specialFile
+            .toString(), fs.exists(specialFile));
+    Assert.assertFalse("File should not be present after successful rename : "
+        + anotherLocation.toString(), fs.exists(anotherLocation));
+
+    Assert.assertTrue("Not listed under parent " + parentPath,
+        contains(fs.listStatus(new Path(parentPath)),
+            fs.makeQualified(specialFile).toString()));
+
+    Assert.assertTrue("Failed to delete " + parentPath,
+        fs.delete(new Path(parentPath), true));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4113ec5f/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
new file mode 100644
index 0000000..3b9e7da
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
@@ -0,0 +1,111 @@
+/*
+ * 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.adl.live;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.UUID;
+
+/**
+ * This class is responsible for testing ContentSummary, ListStatus on
+ * file/folder.
+ */
+public class TestMetadata {
+
+  private FileSystem adlStore;
+  private Path parent;
+
+  public TestMetadata() {
+    parent = new Path("test");
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Assume.assumeTrue(AdlStorageConfiguration.isContractTestEnabled());
+    adlStore = AdlStorageConfiguration.createStorageConnector();
+  }
+
+  @After
+  public void cleanUp() throws Exception {
+    if (AdlStorageConfiguration.isContractTestEnabled()) {
+      adlStore.delete(parent, true);
+    }
+  }
+
+  @Test
+  public void testContentSummaryOnFile() throws IOException {
+    Path child = new Path(UUID.randomUUID().toString());
+    Path testFile = new Path(parent, child);
+    OutputStream out = adlStore.create(testFile);
+
+    for (int i = 0; i < 1024; ++i) {
+      out.write(97);
+    }
+    out.close();
+
+    Assert.assertTrue(adlStore.isFile(testFile));
+    ContentSummary summary = adlStore.getContentSummary(testFile);
+    Assert.assertEquals(1024, summary.getSpaceConsumed());
+    Assert.assertEquals(1, summary.getFileCount());
+    Assert.assertEquals(0, summary.getDirectoryCount());
+    Assert.assertEquals(1024, summary.getLength());
+  }
+
+  @Test
+  public void testContentSummaryOnFolder() throws IOException {
+    Path child = new Path(UUID.randomUUID().toString());
+    Path testFile = new Path(parent, child);
+    OutputStream out = adlStore.create(testFile);
+
+    for (int i = 0; i < 1024; ++i) {
+      out.write(97);
+    }
+    out.close();
+
+    Assert.assertTrue(adlStore.isFile(testFile));
+    ContentSummary summary = adlStore.getContentSummary(parent);
+    Assert.assertEquals(1024, summary.getSpaceConsumed());
+    Assert.assertEquals(1, summary.getFileCount());
+    Assert.assertEquals(1, summary.getDirectoryCount());
+    Assert.assertEquals(1024, summary.getLength());
+  }
+
+  @Test
+  public void listStatusOnFile() throws IOException {
+    Path path = new Path(parent, "a.txt");
+    FileSystem fs = adlStore;
+    fs.createNewFile(path);
+    Assert.assertTrue(fs.isFile(path));
+    FileStatus[] statuses = fs.listStatus(path);
+    Assert
+        .assertEquals(path.makeQualified(fs.getUri(), fs.getWorkingDirectory()),
+            statuses[0].getPath());
+  }
+}
+


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/50] [abbrv] hadoop git commit: HADOOP-13835. Move Google Test Framework code from mapreduce to hadoop-common. Contributed by Varun Vasudev.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/hadoop-common-project/hadoop-common/src/main/native/gtest/gtest-all.cc
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/gtest/gtest-all.cc b/hadoop-common-project/hadoop-common/src/main/native/gtest/gtest-all.cc
new file mode 100644
index 0000000..4f8c08a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/gtest/gtest-all.cc
@@ -0,0 +1,10403 @@
+// Copyright 2008, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Author: mheule@google.com (Markus Heule)
+//
+// Google C++ Testing Framework (Google Test)
+//
+// Sometimes it's desirable to build Google Test by compiling a single file.
+// This file serves this purpose.
+
+// This line ensures that gtest.h can be compiled on its own, even
+// when it's fused.
+#include "gtest/gtest.h"
+
+// The following lines pull in the real gtest *.cc files.
+// Copyright 2005, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Author: wan@google.com (Zhanyong Wan)
+//
+// The Google C++ Testing Framework (Google Test)
+
+// Copyright 2007, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+//
+// Author: wan@google.com (Zhanyong Wan)
+//
+// Utilities for testing Google Test itself and code that uses Google Test
+// (e.g. frameworks built on top of Google Test).
+
+#ifndef GTEST_INCLUDE_GTEST_GTEST_SPI_H_
+#define GTEST_INCLUDE_GTEST_GTEST_SPI_H_
+
+
+namespace testing {
+
+// This helper class can be used to mock out Google Test failure reporting
+// so that we can test Google Test or code that builds on Google Test.
+//
+// An object of this class appends a TestPartResult object to the
+// TestPartResultArray object given in the constructor whenever a Google Test
+// failure is reported. It can either intercept only failures that are
+// generated in the same thread that created this object or it can intercept
+// all generated failures. The scope of this mock object can be controlled with
+// the second argument to the two arguments constructor.
+class GTEST_API_ ScopedFakeTestPartResultReporter
+    : public TestPartResultReporterInterface {
+ public:
+  // The two possible mocking modes of this object.
+  enum InterceptMode {
+    INTERCEPT_ONLY_CURRENT_THREAD,  // Intercepts only thread local failures.
+    INTERCEPT_ALL_THREADS           // Intercepts all failures.
+  };
+
+  // The c'tor sets this object as the test part result reporter used
+  // by Google Test.  The 'result' parameter specifies where to report the
+  // results. This reporter will only catch failures generated in the current
+  // thread. DEPRECATED
+  explicit ScopedFakeTestPartResultReporter(TestPartResultArray* result);
+
+  // Same as above, but you can choose the interception scope of this object.
+  ScopedFakeTestPartResultReporter(InterceptMode intercept_mode,
+                                   TestPartResultArray* result);
+
+  // The d'tor restores the previous test part result reporter.
+  virtual ~ScopedFakeTestPartResultReporter();
+
+  // Appends the TestPartResult object to the TestPartResultArray
+  // received in the constructor.
+  //
+  // This method is from the TestPartResultReporterInterface
+  // interface.
+  virtual void ReportTestPartResult(const TestPartResult& result);
+ private:
+  void Init();
+
+  const InterceptMode intercept_mode_;
+  TestPartResultReporterInterface* old_reporter_;
+  TestPartResultArray* const result_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(ScopedFakeTestPartResultReporter);
+};
+
+namespace internal {
+
+// A helper class for implementing EXPECT_FATAL_FAILURE() and
+// EXPECT_NONFATAL_FAILURE().  Its destructor verifies that the given
+// TestPartResultArray contains exactly one failure that has the given
+// type and contains the given substring.  If that's not the case, a
+// non-fatal failure will be generated.
+class GTEST_API_ SingleFailureChecker {
+ public:
+  // The constructor remembers the arguments.
+  SingleFailureChecker(const TestPartResultArray* results,
+                       TestPartResult::Type type,
+                       const string& substr);
+  ~SingleFailureChecker();
+ private:
+  const TestPartResultArray* const results_;
+  const TestPartResult::Type type_;
+  const string substr_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(SingleFailureChecker);
+};
+
+}  // namespace internal
+
+}  // namespace testing
+
+// A set of macros for testing Google Test assertions or code that's expected
+// to generate Google Test fatal failures.  It verifies that the given
+// statement will cause exactly one fatal Google Test failure with 'substr'
+// being part of the failure message.
+//
+// There are two different versions of this macro. EXPECT_FATAL_FAILURE only
+// affects and considers failures generated in the current thread and
+// EXPECT_FATAL_FAILURE_ON_ALL_THREADS does the same but for all threads.
+//
+// The verification of the assertion is done correctly even when the statement
+// throws an exception or aborts the current function.
+//
+// Known restrictions:
+//   - 'statement' cannot reference local non-static variables or
+//     non-static members of the current object.
+//   - 'statement' cannot return a value.
+//   - You cannot stream a failure message to this macro.
+//
+// Note that even though the implementations of the following two
+// macros are much alike, we cannot refactor them to use a common
+// helper macro, due to some peculiarity in how the preprocessor
+// works.  The AcceptsMacroThatExpandsToUnprotectedComma test in
+// gtest_unittest.cc will fail to compile if we do that.
+#define EXPECT_FATAL_FAILURE(statement, substr) \
+  do { \
+    class GTestExpectFatalFailureHelper {\
+     public:\
+      static void Execute() { statement; }\
+    };\
+    ::testing::TestPartResultArray gtest_failures;\
+    ::testing::internal::SingleFailureChecker gtest_checker(\
+        &gtest_failures, ::testing::TestPartResult::kFatalFailure, (substr));\
+    {\
+      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
+          ::testing::ScopedFakeTestPartResultReporter:: \
+          INTERCEPT_ONLY_CURRENT_THREAD, &gtest_failures);\
+      GTestExpectFatalFailureHelper::Execute();\
+    }\
+  } while (::testing::internal::AlwaysFalse())
+
+#define EXPECT_FATAL_FAILURE_ON_ALL_THREADS(statement, substr) \
+  do { \
+    class GTestExpectFatalFailureHelper {\
+     public:\
+      static void Execute() { statement; }\
+    };\
+    ::testing::TestPartResultArray gtest_failures;\
+    ::testing::internal::SingleFailureChecker gtest_checker(\
+        &gtest_failures, ::testing::TestPartResult::kFatalFailure, (substr));\
+    {\
+      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
+          ::testing::ScopedFakeTestPartResultReporter:: \
+          INTERCEPT_ALL_THREADS, &gtest_failures);\
+      GTestExpectFatalFailureHelper::Execute();\
+    }\
+  } while (::testing::internal::AlwaysFalse())
+
+// A macro for testing Google Test assertions or code that's expected to
+// generate Google Test non-fatal failures.  It asserts that the given
+// statement will cause exactly one non-fatal Google Test failure with 'substr'
+// being part of the failure message.
+//
+// There are two different versions of this macro. EXPECT_NONFATAL_FAILURE only
+// affects and considers failures generated in the current thread and
+// EXPECT_NONFATAL_FAILURE_ON_ALL_THREADS does the same but for all threads.
+//
+// 'statement' is allowed to reference local variables and members of
+// the current object.
+//
+// The verification of the assertion is done correctly even when the statement
+// throws an exception or aborts the current function.
+//
+// Known restrictions:
+//   - You cannot stream a failure message to this macro.
+//
+// Note that even though the implementations of the following two
+// macros are much alike, we cannot refactor them to use a common
+// helper macro, due to some peculiarity in how the preprocessor
+// works.  If we do that, the code won't compile when the user gives
+// EXPECT_NONFATAL_FAILURE() a statement that contains a macro that
+// expands to code containing an unprotected comma.  The
+// AcceptsMacroThatExpandsToUnprotectedComma test in gtest_unittest.cc
+// catches that.
+//
+// For the same reason, we have to write
+//   if (::testing::internal::AlwaysTrue()) { statement; }
+// instead of
+//   GTEST_SUPPRESS_UNREACHABLE_CODE_WARNING_BELOW_(statement)
+// to avoid an MSVC warning on unreachable code.
+#define EXPECT_NONFATAL_FAILURE(statement, substr) \
+  do {\
+    ::testing::TestPartResultArray gtest_failures;\
+    ::testing::internal::SingleFailureChecker gtest_checker(\
+        &gtest_failures, ::testing::TestPartResult::kNonFatalFailure, \
+        (substr));\
+    {\
+      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
+          ::testing::ScopedFakeTestPartResultReporter:: \
+          INTERCEPT_ONLY_CURRENT_THREAD, &gtest_failures);\
+      if (::testing::internal::AlwaysTrue()) { statement; }\
+    }\
+  } while (::testing::internal::AlwaysFalse())
+
+#define EXPECT_NONFATAL_FAILURE_ON_ALL_THREADS(statement, substr) \
+  do {\
+    ::testing::TestPartResultArray gtest_failures;\
+    ::testing::internal::SingleFailureChecker gtest_checker(\
+        &gtest_failures, ::testing::TestPartResult::kNonFatalFailure, \
+        (substr));\
+    {\
+      ::testing::ScopedFakeTestPartResultReporter gtest_reporter(\
+          ::testing::ScopedFakeTestPartResultReporter::INTERCEPT_ALL_THREADS, \
+          &gtest_failures);\
+      if (::testing::internal::AlwaysTrue()) { statement; }\
+    }\
+  } while (::testing::internal::AlwaysFalse())
+
+#endif  // GTEST_INCLUDE_GTEST_GTEST_SPI_H_
+
+#include <ctype.h>
+#include <math.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <time.h>
+#include <wchar.h>
+#include <wctype.h>
+
+#include <algorithm>
+#include <iomanip>
+#include <limits>
+#include <list>
+#include <map>
+#include <ostream>  // NOLINT
+#include <sstream>
+#include <vector>
+
+#if GTEST_OS_LINUX
+
+// TODO(kenton@google.com): Use autoconf to detect availability of
+// gettimeofday().
+# define GTEST_HAS_GETTIMEOFDAY_ 1
+
+# include <fcntl.h>  // NOLINT
+# include <limits.h>  // NOLINT
+# include <sched.h>  // NOLINT
+// Declares vsnprintf().  This header is not available on Windows.
+# include <strings.h>  // NOLINT
+# include <sys/mman.h>  // NOLINT
+# include <sys/time.h>  // NOLINT
+# include <unistd.h>  // NOLINT
+# include <string>
+
+#elif GTEST_OS_SYMBIAN
+# define GTEST_HAS_GETTIMEOFDAY_ 1
+# include <sys/time.h>  // NOLINT
+
+#elif GTEST_OS_ZOS
+# define GTEST_HAS_GETTIMEOFDAY_ 1
+# include <sys/time.h>  // NOLINT
+
+// On z/OS we additionally need strings.h for strcasecmp.
+# include <strings.h>  // NOLINT
+
+#elif GTEST_OS_WINDOWS_MOBILE  // We are on Windows CE.
+
+# include <windows.h>  // NOLINT
+# undef min
+
+#elif GTEST_OS_WINDOWS  // We are on Windows proper.
+
+# include <io.h>  // NOLINT
+# include <sys/timeb.h>  // NOLINT
+# include <sys/types.h>  // NOLINT
+# include <sys/stat.h>  // NOLINT
+
+# if GTEST_OS_WINDOWS_MINGW
+// MinGW has gettimeofday() but not _ftime64().
+// TODO(kenton@google.com): Use autoconf to detect availability of
+//   gettimeofday().
+// TODO(kenton@google.com): There are other ways to get the time on
+//   Windows, like GetTickCount() or GetSystemTimeAsFileTime().  MinGW
+//   supports these.  consider using them instead.
+#  define GTEST_HAS_GETTIMEOFDAY_ 1
+#  include <sys/time.h>  // NOLINT
+# endif  // GTEST_OS_WINDOWS_MINGW
+
+// cpplint thinks that the header is already included, so we want to
+// silence it.
+# include <windows.h>  // NOLINT
+# undef min
+
+#else
+
+// Assume other platforms have gettimeofday().
+// TODO(kenton@google.com): Use autoconf to detect availability of
+//   gettimeofday().
+# define GTEST_HAS_GETTIMEOFDAY_ 1
+
+// cpplint thinks that the header is already included, so we want to
+// silence it.
+# include <sys/time.h>  // NOLINT
+# include <unistd.h>  // NOLINT
+
+#endif  // GTEST_OS_LINUX
+
+#if GTEST_HAS_EXCEPTIONS
+# include <stdexcept>
+#endif
+
+#if GTEST_CAN_STREAM_RESULTS_
+# include <arpa/inet.h>  // NOLINT
+# include <netdb.h>  // NOLINT
+# include <sys/socket.h>  // NOLINT
+# include <sys/types.h>  // NOLINT
+#endif
+
+// Indicates that this translation unit is part of Google Test's
+// implementation.  It must come before gtest-internal-inl.h is
+// included, or there will be a compiler error.  This trick is to
+// prevent a user from accidentally including gtest-internal-inl.h in
+// his code.
+#define GTEST_IMPLEMENTATION_ 1
+// Copyright 2005, Google Inc.
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+// Utility functions and classes used by the Google C++ testing framework.
+//
+// Author: wan@google.com (Zhanyong Wan)
+//
+// This file contains purely Google Test's internal implementation.  Please
+// DO NOT #INCLUDE IT IN A USER PROGRAM.
+
+#ifndef GTEST_SRC_GTEST_INTERNAL_INL_H_
+#define GTEST_SRC_GTEST_INTERNAL_INL_H_
+
+// GTEST_IMPLEMENTATION_ is defined to 1 iff the current translation unit is
+// part of Google Test's implementation; otherwise it's undefined.
+#if !GTEST_IMPLEMENTATION_
+// If this file is included from the user's code, just say no.
+# error "gtest-internal-inl.h is part of Google Test's internal implementation."
+# error "It must not be included except by Google Test itself."
+#endif  // GTEST_IMPLEMENTATION_
+
+#ifndef _WIN32_WCE
+# include <errno.h>
+#endif  // !_WIN32_WCE
+#include <stddef.h>
+#include <stdlib.h>  // For strtoll/_strtoul64/malloc/free.
+#include <string.h>  // For memmove.
+
+#include <algorithm>
+#include <string>
+#include <vector>
+
+
+#if GTEST_CAN_STREAM_RESULTS_
+# include <arpa/inet.h>  // NOLINT
+# include <netdb.h>  // NOLINT
+#endif
+
+#if GTEST_OS_WINDOWS
+# include <windows.h>  // NOLINT
+#endif  // GTEST_OS_WINDOWS
+
+
+namespace testing {
+
+// Declares the flags.
+//
+// We don't want the users to modify this flag in the code, but want
+// Google Test's own unit tests to be able to access it. Therefore we
+// declare it here as opposed to in gtest.h.
+GTEST_DECLARE_bool_(death_test_use_fork);
+
+namespace internal {
+
+// The value of GetTestTypeId() as seen from within the Google Test
+// library.  This is solely for testing GetTestTypeId().
+GTEST_API_ extern const TypeId kTestTypeIdInGoogleTest;
+
+// Names of the flags (needed for parsing Google Test flags).
+const char kAlsoRunDisabledTestsFlag[] = "also_run_disabled_tests";
+const char kBreakOnFailureFlag[] = "break_on_failure";
+const char kCatchExceptionsFlag[] = "catch_exceptions";
+const char kColorFlag[] = "color";
+const char kFilterFlag[] = "filter";
+const char kListTestsFlag[] = "list_tests";
+const char kOutputFlag[] = "output";
+const char kPrintTimeFlag[] = "print_time";
+const char kRandomSeedFlag[] = "random_seed";
+const char kRepeatFlag[] = "repeat";
+const char kShuffleFlag[] = "shuffle";
+const char kStackTraceDepthFlag[] = "stack_trace_depth";
+const char kStreamResultToFlag[] = "stream_result_to";
+const char kThrowOnFailureFlag[] = "throw_on_failure";
+const char kFlagfileFlag[] = "flagfile";
+
+// A valid random seed must be in [1, kMaxRandomSeed].
+const int kMaxRandomSeed = 99999;
+
+// g_help_flag is true iff the --help flag or an equivalent form is
+// specified on the command line.
+GTEST_API_ extern bool g_help_flag;
+
+// Returns the current time in milliseconds.
+GTEST_API_ TimeInMillis GetTimeInMillis();
+
+// Returns true iff Google Test should use colors in the output.
+GTEST_API_ bool ShouldUseColor(bool stdout_is_tty);
+
+// Formats the given time in milliseconds as seconds.
+GTEST_API_ std::string FormatTimeInMillisAsSeconds(TimeInMillis ms);
+
+// Converts the given time in milliseconds to a date string in the ISO 8601
+// format, without the timezone information.  N.B.: due to the use the
+// non-reentrant localtime() function, this function is not thread safe.  Do
+// not use it in any code that can be called from multiple threads.
+GTEST_API_ std::string FormatEpochTimeInMillisAsIso8601(TimeInMillis ms);
+
+// Parses a string for an Int32 flag, in the form of "--flag=value".
+//
+// On success, stores the value of the flag in *value, and returns
+// true.  On failure, returns false without changing *value.
+GTEST_API_ bool ParseInt32Flag(
+    const char* str, const char* flag, Int32* value);
+
+// Returns a random seed in range [1, kMaxRandomSeed] based on the
+// given --gtest_random_seed flag value.
+inline int GetRandomSeedFromFlag(Int32 random_seed_flag) {
+  const unsigned int raw_seed = (random_seed_flag == 0) ?
+      static_cast<unsigned int>(GetTimeInMillis()) :
+      static_cast<unsigned int>(random_seed_flag);
+
+  // Normalizes the actual seed to range [1, kMaxRandomSeed] such that
+  // it's easy to type.
+  const int normalized_seed =
+      static_cast<int>((raw_seed - 1U) %
+                       static_cast<unsigned int>(kMaxRandomSeed)) + 1;
+  return normalized_seed;
+}
+
+// Returns the first valid random seed after 'seed'.  The behavior is
+// undefined if 'seed' is invalid.  The seed after kMaxRandomSeed is
+// considered to be 1.
+inline int GetNextRandomSeed(int seed) {
+  GTEST_CHECK_(1 <= seed && seed <= kMaxRandomSeed)
+      << "Invalid random seed " << seed << " - must be in [1, "
+      << kMaxRandomSeed << "].";
+  const int next_seed = seed + 1;
+  return (next_seed > kMaxRandomSeed) ? 1 : next_seed;
+}
+
+// This class saves the values of all Google Test flags in its c'tor, and
+// restores them in its d'tor.
+class GTestFlagSaver {
+ public:
+  // The c'tor.
+  GTestFlagSaver() {
+    also_run_disabled_tests_ = GTEST_FLAG(also_run_disabled_tests);
+    break_on_failure_ = GTEST_FLAG(break_on_failure);
+    catch_exceptions_ = GTEST_FLAG(catch_exceptions);
+    color_ = GTEST_FLAG(color);
+    death_test_style_ = GTEST_FLAG(death_test_style);
+    death_test_use_fork_ = GTEST_FLAG(death_test_use_fork);
+    filter_ = GTEST_FLAG(filter);
+    internal_run_death_test_ = GTEST_FLAG(internal_run_death_test);
+    list_tests_ = GTEST_FLAG(list_tests);
+    output_ = GTEST_FLAG(output);
+    print_time_ = GTEST_FLAG(print_time);
+    random_seed_ = GTEST_FLAG(random_seed);
+    repeat_ = GTEST_FLAG(repeat);
+    shuffle_ = GTEST_FLAG(shuffle);
+    stack_trace_depth_ = GTEST_FLAG(stack_trace_depth);
+    stream_result_to_ = GTEST_FLAG(stream_result_to);
+    throw_on_failure_ = GTEST_FLAG(throw_on_failure);
+  }
+
+  // The d'tor is not virtual.  DO NOT INHERIT FROM THIS CLASS.
+  ~GTestFlagSaver() {
+    GTEST_FLAG(also_run_disabled_tests) = also_run_disabled_tests_;
+    GTEST_FLAG(break_on_failure) = break_on_failure_;
+    GTEST_FLAG(catch_exceptions) = catch_exceptions_;
+    GTEST_FLAG(color) = color_;
+    GTEST_FLAG(death_test_style) = death_test_style_;
+    GTEST_FLAG(death_test_use_fork) = death_test_use_fork_;
+    GTEST_FLAG(filter) = filter_;
+    GTEST_FLAG(internal_run_death_test) = internal_run_death_test_;
+    GTEST_FLAG(list_tests) = list_tests_;
+    GTEST_FLAG(output) = output_;
+    GTEST_FLAG(print_time) = print_time_;
+    GTEST_FLAG(random_seed) = random_seed_;
+    GTEST_FLAG(repeat) = repeat_;
+    GTEST_FLAG(shuffle) = shuffle_;
+    GTEST_FLAG(stack_trace_depth) = stack_trace_depth_;
+    GTEST_FLAG(stream_result_to) = stream_result_to_;
+    GTEST_FLAG(throw_on_failure) = throw_on_failure_;
+  }
+
+ private:
+  // Fields for saving the original values of flags.
+  bool also_run_disabled_tests_;
+  bool break_on_failure_;
+  bool catch_exceptions_;
+  std::string color_;
+  std::string death_test_style_;
+  bool death_test_use_fork_;
+  std::string filter_;
+  std::string internal_run_death_test_;
+  bool list_tests_;
+  std::string output_;
+  bool print_time_;
+  internal::Int32 random_seed_;
+  internal::Int32 repeat_;
+  bool shuffle_;
+  internal::Int32 stack_trace_depth_;
+  std::string stream_result_to_;
+  bool throw_on_failure_;
+} GTEST_ATTRIBUTE_UNUSED_;
+
+// Converts a Unicode code point to a narrow string in UTF-8 encoding.
+// code_point parameter is of type UInt32 because wchar_t may not be
+// wide enough to contain a code point.
+// If the code_point is not a valid Unicode code point
+// (i.e. outside of Unicode range U+0 to U+10FFFF) it will be converted
+// to "(Invalid Unicode 0xXXXXXXXX)".
+GTEST_API_ std::string CodePointToUtf8(UInt32 code_point);
+
+// Converts a wide string to a narrow string in UTF-8 encoding.
+// The wide string is assumed to have the following encoding:
+//   UTF-16 if sizeof(wchar_t) == 2 (on Windows, Cygwin, Symbian OS)
+//   UTF-32 if sizeof(wchar_t) == 4 (on Linux)
+// Parameter str points to a null-terminated wide string.
+// Parameter num_chars may additionally limit the number
+// of wchar_t characters processed. -1 is used when the entire string
+// should be processed.
+// If the string contains code points that are not valid Unicode code points
+// (i.e. outside of Unicode range U+0 to U+10FFFF) they will be output
+// as '(Invalid Unicode 0xXXXXXXXX)'. If the string is in UTF16 encoding
+// and contains invalid UTF-16 surrogate pairs, values in those pairs
+// will be encoded as individual Unicode characters from Basic Normal Plane.
+GTEST_API_ std::string WideStringToUtf8(const wchar_t* str, int num_chars);
+
+// Reads the GTEST_SHARD_STATUS_FILE environment variable, and creates the file
+// if the variable is present. If a file already exists at this location, this
+// function will write over it. If the variable is present, but the file cannot
+// be created, prints an error and exits.
+void WriteToShardStatusFileIfNeeded();
+
+// Checks whether sharding is enabled by examining the relevant
+// environment variable values. If the variables are present,
+// but inconsistent (e.g., shard_index >= total_shards), prints
+// an error and exits. If in_subprocess_for_death_test, sharding is
+// disabled because it must only be applied to the original test
+// process. Otherwise, we could filter out death tests we intended to execute.
+GTEST_API_ bool ShouldShard(const char* total_shards_str,
+                            const char* shard_index_str,
+                            bool in_subprocess_for_death_test);
+
+// Parses the environment variable var as an Int32. If it is unset,
+// returns default_val. If it is not an Int32, prints an error and
+// and aborts.
+GTEST_API_ Int32 Int32FromEnvOrDie(const char* env_var, Int32 default_val);
+
+// Given the total number of shards, the shard index, and the test id,
+// returns true iff the test should be run on this shard. The test id is
+// some arbitrary but unique non-negative integer assigned to each test
+// method. Assumes that 0 <= shard_index < total_shards.
+GTEST_API_ bool ShouldRunTestOnShard(
+    int total_shards, int shard_index, int test_id);
+
+// STL container utilities.
+
+// Returns the number of elements in the given container that satisfy
+// the given predicate.
+template <class Container, typename Predicate>
+inline int CountIf(const Container& c, Predicate predicate) {
+  // Implemented as an explicit loop since std::count_if() in libCstd on
+  // Solaris has a non-standard signature.
+  int count = 0;
+  for (typename Container::const_iterator it = c.begin(); it != c.end(); ++it) {
+    if (predicate(*it))
+      ++count;
+  }
+  return count;
+}
+
+// Applies a function/functor to each element in the container.
+template <class Container, typename Functor>
+void ForEach(const Container& c, Functor functor) {
+  std::for_each(c.begin(), c.end(), functor);
+}
+
+// Returns the i-th element of the vector, or default_value if i is not
+// in range [0, v.size()).
+template <typename E>
+inline E GetElementOr(const std::vector<E>& v, int i, E default_value) {
+  return (i < 0 || i >= static_cast<int>(v.size())) ? default_value : v[i];
+}
+
+// Performs an in-place shuffle of a range of the vector's elements.
+// 'begin' and 'end' are element indices as an STL-style range;
+// i.e. [begin, end) are shuffled, where 'end' == size() means to
+// shuffle to the end of the vector.
+template <typename E>
+void ShuffleRange(internal::Random* random, int begin, int end,
+                  std::vector<E>* v) {
+  const int size = static_cast<int>(v->size());
+  GTEST_CHECK_(0 <= begin && begin <= size)
+      << "Invalid shuffle range start " << begin << ": must be in range [0, "
+      << size << "].";
+  GTEST_CHECK_(begin <= end && end <= size)
+      << "Invalid shuffle range finish " << end << ": must be in range ["
+      << begin << ", " << size << "].";
+
+  // Fisher-Yates shuffle, from
+  // http://en.wikipedia.org/wiki/Fisher-Yates_shuffle
+  for (int range_width = end - begin; range_width >= 2; range_width--) {
+    const int last_in_range = begin + range_width - 1;
+    const int selected = begin + random->Generate(range_width);
+    std::swap((*v)[selected], (*v)[last_in_range]);
+  }
+}
+
+// Performs an in-place shuffle of the vector's elements.
+template <typename E>
+inline void Shuffle(internal::Random* random, std::vector<E>* v) {
+  ShuffleRange(random, 0, static_cast<int>(v->size()), v);
+}
+
+// A function for deleting an object.  Handy for being used as a
+// functor.
+template <typename T>
+static void Delete(T* x) {
+  delete x;
+}
+
+// A predicate that checks the key of a TestProperty against a known key.
+//
+// TestPropertyKeyIs is copyable.
+class TestPropertyKeyIs {
+ public:
+  // Constructor.
+  //
+  // TestPropertyKeyIs has NO default constructor.
+  explicit TestPropertyKeyIs(const std::string& key) : key_(key) {}
+
+  // Returns true iff the test name of test property matches on key_.
+  bool operator()(const TestProperty& test_property) const {
+    return test_property.key() == key_;
+  }
+
+ private:
+  std::string key_;
+};
+
+// Class UnitTestOptions.
+//
+// This class contains functions for processing options the user
+// specifies when running the tests.  It has only static members.
+//
+// In most cases, the user can specify an option using either an
+// environment variable or a command line flag.  E.g. you can set the
+// test filter using either GTEST_FILTER or --gtest_filter.  If both
+// the variable and the flag are present, the latter overrides the
+// former.
+class GTEST_API_ UnitTestOptions {
+ public:
+  // Functions for processing the gtest_output flag.
+
+  // Returns the output format, or "" for normal printed output.
+  static std::string GetOutputFormat();
+
+  // Returns the absolute path of the requested output file, or the
+  // default (test_detail.xml in the original working directory) if
+  // none was explicitly specified.
+  static std::string GetAbsolutePathToOutputFile();
+
+  // Functions for processing the gtest_filter flag.
+
+  // Returns true iff the wildcard pattern matches the string.  The
+  // first ':' or '\0' character in pattern marks the end of it.
+  //
+  // This recursive algorithm isn't very efficient, but is clear and
+  // works well enough for matching test names, which are short.
+  static bool PatternMatchesString(const char *pattern, const char *str);
+
+  // Returns true iff the user-specified filter matches the test case
+  // name and the test name.
+  static bool FilterMatchesTest(const std::string &test_case_name,
+                                const std::string &test_name);
+
+#if GTEST_OS_WINDOWS
+  // Function for supporting the gtest_catch_exception flag.
+
+  // Returns EXCEPTION_EXECUTE_HANDLER if Google Test should handle the
+  // given SEH exception, or EXCEPTION_CONTINUE_SEARCH otherwise.
+  // This function is useful as an __except condition.
+  static int GTestShouldProcessSEH(DWORD exception_code);
+#endif  // GTEST_OS_WINDOWS
+
+  // Returns true if "name" matches the ':' separated list of glob-style
+  // filters in "filter".
+  static bool MatchesFilter(const std::string& name, const char* filter);
+};
+
+// Returns the current application's name, removing directory path if that
+// is present.  Used by UnitTestOptions::GetOutputFile.
+GTEST_API_ FilePath GetCurrentExecutableName();
+
+// The role interface for getting the OS stack trace as a string.
+class OsStackTraceGetterInterface {
+ public:
+  OsStackTraceGetterInterface() {}
+  virtual ~OsStackTraceGetterInterface() {}
+
+  // Returns the current OS stack trace as an std::string.  Parameters:
+  //
+  //   max_depth  - the maximum number of stack frames to be included
+  //                in the trace.
+  //   skip_count - the number of top frames to be skipped; doesn't count
+  //                against max_depth.
+  virtual string CurrentStackTrace(int max_depth, int skip_count) = 0;
+
+  // UponLeavingGTest() should be called immediately before Google Test calls
+  // user code. It saves some information about the current stack that
+  // CurrentStackTrace() will use to find and hide Google Test stack frames.
+  virtual void UponLeavingGTest() = 0;
+
+  // This string is inserted in place of stack frames that are part of
+  // Google Test's implementation.
+  static const char* const kElidedFramesMarker;
+
+ private:
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(OsStackTraceGetterInterface);
+};
+
+// A working implementation of the OsStackTraceGetterInterface interface.
+class OsStackTraceGetter : public OsStackTraceGetterInterface {
+ public:
+  OsStackTraceGetter() {}
+
+  virtual string CurrentStackTrace(int max_depth, int skip_count);
+  virtual void UponLeavingGTest();
+
+ private:
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(OsStackTraceGetter);
+};
+
+// Information about a Google Test trace point.
+struct TraceInfo {
+  const char* file;
+  int line;
+  std::string message;
+};
+
+// This is the default global test part result reporter used in UnitTestImpl.
+// This class should only be used by UnitTestImpl.
+class DefaultGlobalTestPartResultReporter
+  : public TestPartResultReporterInterface {
+ public:
+  explicit DefaultGlobalTestPartResultReporter(UnitTestImpl* unit_test);
+  // Implements the TestPartResultReporterInterface. Reports the test part
+  // result in the current test.
+  virtual void ReportTestPartResult(const TestPartResult& result);
+
+ private:
+  UnitTestImpl* const unit_test_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(DefaultGlobalTestPartResultReporter);
+};
+
+// This is the default per thread test part result reporter used in
+// UnitTestImpl. This class should only be used by UnitTestImpl.
+class DefaultPerThreadTestPartResultReporter
+    : public TestPartResultReporterInterface {
+ public:
+  explicit DefaultPerThreadTestPartResultReporter(UnitTestImpl* unit_test);
+  // Implements the TestPartResultReporterInterface. The implementation just
+  // delegates to the current global test part result reporter of *unit_test_.
+  virtual void ReportTestPartResult(const TestPartResult& result);
+
+ private:
+  UnitTestImpl* const unit_test_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(DefaultPerThreadTestPartResultReporter);
+};
+
+// The private implementation of the UnitTest class.  We don't protect
+// the methods under a mutex, as this class is not accessible by a
+// user and the UnitTest class that delegates work to this class does
+// proper locking.
+class GTEST_API_ UnitTestImpl {
+ public:
+  explicit UnitTestImpl(UnitTest* parent);
+  virtual ~UnitTestImpl();
+
+  // There are two different ways to register your own TestPartResultReporter.
+  // You can register your own repoter to listen either only for test results
+  // from the current thread or for results from all threads.
+  // By default, each per-thread test result repoter just passes a new
+  // TestPartResult to the global test result reporter, which registers the
+  // test part result for the currently running test.
+
+  // Returns the global test part result reporter.
+  TestPartResultReporterInterface* GetGlobalTestPartResultReporter();
+
+  // Sets the global test part result reporter.
+  void SetGlobalTestPartResultReporter(
+      TestPartResultReporterInterface* reporter);
+
+  // Returns the test part result reporter for the current thread.
+  TestPartResultReporterInterface* GetTestPartResultReporterForCurrentThread();
+
+  // Sets the test part result reporter for the current thread.
+  void SetTestPartResultReporterForCurrentThread(
+      TestPartResultReporterInterface* reporter);
+
+  // Gets the number of successful test cases.
+  int successful_test_case_count() const;
+
+  // Gets the number of failed test cases.
+  int failed_test_case_count() const;
+
+  // Gets the number of all test cases.
+  int total_test_case_count() const;
+
+  // Gets the number of all test cases that contain at least one test
+  // that should run.
+  int test_case_to_run_count() const;
+
+  // Gets the number of successful tests.
+  int successful_test_count() const;
+
+  // Gets the number of failed tests.
+  int failed_test_count() const;
+
+  // Gets the number of disabled tests that will be reported in the XML report.
+  int reportable_disabled_test_count() const;
+
+  // Gets the number of disabled tests.
+  int disabled_test_count() const;
+
+  // Gets the number of tests to be printed in the XML report.
+  int reportable_test_count() const;
+
+  // Gets the number of all tests.
+  int total_test_count() const;
+
+  // Gets the number of tests that should run.
+  int test_to_run_count() const;
+
+  // Gets the time of the test program start, in ms from the start of the
+  // UNIX epoch.
+  TimeInMillis start_timestamp() const { return start_timestamp_; }
+
+  // Gets the elapsed time, in milliseconds.
+  TimeInMillis elapsed_time() const { return elapsed_time_; }
+
+  // Returns true iff the unit test passed (i.e. all test cases passed).
+  bool Passed() const { return !Failed(); }
+
+  // Returns true iff the unit test failed (i.e. some test case failed
+  // or something outside of all tests failed).
+  bool Failed() const {
+    return failed_test_case_count() > 0 || ad_hoc_test_result()->Failed();
+  }
+
+  // Gets the i-th test case among all the test cases. i can range from 0 to
+  // total_test_case_count() - 1. If i is not in that range, returns NULL.
+  const TestCase* GetTestCase(int i) const {
+    const int index = GetElementOr(test_case_indices_, i, -1);
+    return index < 0 ? NULL : test_cases_[i];
+  }
+
+  // Gets the i-th test case among all the test cases. i can range from 0 to
+  // total_test_case_count() - 1. If i is not in that range, returns NULL.
+  TestCase* GetMutableTestCase(int i) {
+    const int index = GetElementOr(test_case_indices_, i, -1);
+    return index < 0 ? NULL : test_cases_[index];
+  }
+
+  // Provides access to the event listener list.
+  TestEventListeners* listeners() { return &listeners_; }
+
+  // Returns the TestResult for the test that's currently running, or
+  // the TestResult for the ad hoc test if no test is running.
+  TestResult* current_test_result();
+
+  // Returns the TestResult for the ad hoc test.
+  const TestResult* ad_hoc_test_result() const { return &ad_hoc_test_result_; }
+
+  // Sets the OS stack trace getter.
+  //
+  // Does nothing if the input and the current OS stack trace getter
+  // are the same; otherwise, deletes the old getter and makes the
+  // input the current getter.
+  void set_os_stack_trace_getter(OsStackTraceGetterInterface* getter);
+
+  // Returns the current OS stack trace getter if it is not NULL;
+  // otherwise, creates an OsStackTraceGetter, makes it the current
+  // getter, and returns it.
+  OsStackTraceGetterInterface* os_stack_trace_getter();
+
+  // Returns the current OS stack trace as an std::string.
+  //
+  // The maximum number of stack frames to be included is specified by
+  // the gtest_stack_trace_depth flag.  The skip_count parameter
+  // specifies the number of top frames to be skipped, which doesn't
+  // count against the number of frames to be included.
+  //
+  // For example, if Foo() calls Bar(), which in turn calls
+  // CurrentOsStackTraceExceptTop(1), Foo() will be included in the
+  // trace but Bar() and CurrentOsStackTraceExceptTop() won't.
+  std::string CurrentOsStackTraceExceptTop(int skip_count) GTEST_NO_INLINE_;
+
+  // Finds and returns a TestCase with the given name.  If one doesn't
+  // exist, creates one and returns it.
+  //
+  // Arguments:
+  //
+  //   test_case_name: name of the test case
+  //   type_param:     the name of the test's type parameter, or NULL if
+  //                   this is not a typed or a type-parameterized test.
+  //   set_up_tc:      pointer to the function that sets up the test case
+  //   tear_down_tc:   pointer to the function that tears down the test case
+  TestCase* GetTestCase(const char* test_case_name,
+                        const char* type_param,
+                        Test::SetUpTestCaseFunc set_up_tc,
+                        Test::TearDownTestCaseFunc tear_down_tc);
+
+  // Adds a TestInfo to the unit test.
+  //
+  // Arguments:
+  //
+  //   set_up_tc:    pointer to the function that sets up the test case
+  //   tear_down_tc: pointer to the function that tears down the test case
+  //   test_info:    the TestInfo object
+  void AddTestInfo(Test::SetUpTestCaseFunc set_up_tc,
+                   Test::TearDownTestCaseFunc tear_down_tc,
+                   TestInfo* test_info) {
+    // In order to support thread-safe death tests, we need to
+    // remember the original working directory when the test program
+    // was first invoked.  We cannot do this in RUN_ALL_TESTS(), as
+    // the user may have changed the current directory before calling
+    // RUN_ALL_TESTS().  Therefore we capture the current directory in
+    // AddTestInfo(), which is called to register a TEST or TEST_F
+    // before main() is reached.
+    if (original_working_dir_.IsEmpty()) {
+      original_working_dir_.Set(FilePath::GetCurrentDir());
+      GTEST_CHECK_(!original_working_dir_.IsEmpty())
+          << "Failed to get the current working directory.";
+    }
+
+    GetTestCase(test_info->test_case_name(),
+                test_info->type_param(),
+                set_up_tc,
+                tear_down_tc)->AddTestInfo(test_info);
+  }
+
+#if GTEST_HAS_PARAM_TEST
+  // Returns ParameterizedTestCaseRegistry object used to keep track of
+  // value-parameterized tests and instantiate and register them.
+  internal::ParameterizedTestCaseRegistry& parameterized_test_registry() {
+    return parameterized_test_registry_;
+  }
+#endif  // GTEST_HAS_PARAM_TEST
+
+  // Sets the TestCase object for the test that's currently running.
+  void set_current_test_case(TestCase* a_current_test_case) {
+    current_test_case_ = a_current_test_case;
+  }
+
+  // Sets the TestInfo object for the test that's currently running.  If
+  // current_test_info is NULL, the assertion results will be stored in
+  // ad_hoc_test_result_.
+  void set_current_test_info(TestInfo* a_current_test_info) {
+    current_test_info_ = a_current_test_info;
+  }
+
+  // Registers all parameterized tests defined using TEST_P and
+  // INSTANTIATE_TEST_CASE_P, creating regular tests for each test/parameter
+  // combination. This method can be called more then once; it has guards
+  // protecting from registering the tests more then once.  If
+  // value-parameterized tests are disabled, RegisterParameterizedTests is
+  // present but does nothing.
+  void RegisterParameterizedTests();
+
+  // Runs all tests in this UnitTest object, prints the result, and
+  // returns true if all tests are successful.  If any exception is
+  // thrown during a test, this test is considered to be failed, but
+  // the rest of the tests will still be run.
+  bool RunAllTests();
+
+  // Clears the results of all tests, except the ad hoc tests.
+  void ClearNonAdHocTestResult() {
+    ForEach(test_cases_, TestCase::ClearTestCaseResult);
+  }
+
+  // Clears the results of ad-hoc test assertions.
+  void ClearAdHocTestResult() {
+    ad_hoc_test_result_.Clear();
+  }
+
+  // Adds a TestProperty to the current TestResult object when invoked in a
+  // context of a test or a test case, or to the global property set. If the
+  // result already contains a property with the same key, the value will be
+  // updated.
+  void RecordProperty(const TestProperty& test_property);
+
+  enum ReactionToSharding {
+    HONOR_SHARDING_PROTOCOL,
+    IGNORE_SHARDING_PROTOCOL
+  };
+
+  // Matches the full name of each test against the user-specified
+  // filter to decide whether the test should run, then records the
+  // result in each TestCase and TestInfo object.
+  // If shard_tests == HONOR_SHARDING_PROTOCOL, further filters tests
+  // based on sharding variables in the environment.
+  // Returns the number of tests that should run.
+  int FilterTests(ReactionToSharding shard_tests);
+
+  // Prints the names of the tests matching the user-specified filter flag.
+  void ListTestsMatchingFilter();
+
+  const TestCase* current_test_case() const { return current_test_case_; }
+  TestInfo* current_test_info() { return current_test_info_; }
+  const TestInfo* current_test_info() const { return current_test_info_; }
+
+  // Returns the vector of environments that need to be set-up/torn-down
+  // before/after the tests are run.
+  std::vector<Environment*>& environments() { return environments_; }
+
+  // Getters for the per-thread Google Test trace stack.
+  std::vector<TraceInfo>& gtest_trace_stack() {
+    return *(gtest_trace_stack_.pointer());
+  }
+  const std::vector<TraceInfo>& gtest_trace_stack() const {
+    return gtest_trace_stack_.get();
+  }
+
+#if GTEST_HAS_DEATH_TEST
+  void InitDeathTestSubprocessControlInfo() {
+    internal_run_death_test_flag_.reset(ParseInternalRunDeathTestFlag());
+  }
+  // Returns a pointer to the parsed --gtest_internal_run_death_test
+  // flag, or NULL if that flag was not specified.
+  // This information is useful only in a death test child process.
+  // Must not be called before a call to InitGoogleTest.
+  const InternalRunDeathTestFlag* internal_run_death_test_flag() const {
+    return internal_run_death_test_flag_.get();
+  }
+
+  // Returns a pointer to the current death test factory.
+  internal::DeathTestFactory* death_test_factory() {
+    return death_test_factory_.get();
+  }
+
+  void SuppressTestEventsIfInSubprocess();
+
+  friend class ReplaceDeathTestFactory;
+#endif  // GTEST_HAS_DEATH_TEST
+
+  // Initializes the event listener performing XML output as specified by
+  // UnitTestOptions. Must not be called before InitGoogleTest.
+  void ConfigureXmlOutput();
+
+#if GTEST_CAN_STREAM_RESULTS_
+  // Initializes the event listener for streaming test results to a socket.
+  // Must not be called before InitGoogleTest.
+  void ConfigureStreamingOutput();
+#endif
+
+  // Performs initialization dependent upon flag values obtained in
+  // ParseGoogleTestFlagsOnly.  Is called from InitGoogleTest after the call to
+  // ParseGoogleTestFlagsOnly.  In case a user neglects to call InitGoogleTest
+  // this function is also called from RunAllTests.  Since this function can be
+  // called more than once, it has to be idempotent.
+  void PostFlagParsingInit();
+
+  // Gets the random seed used at the start of the current test iteration.
+  int random_seed() const { return random_seed_; }
+
+  // Gets the random number generator.
+  internal::Random* random() { return &random_; }
+
+  // Shuffles all test cases, and the tests within each test case,
+  // making sure that death tests are still run first.
+  void ShuffleTests();
+
+  // Restores the test cases and tests to their order before the first shuffle.
+  void UnshuffleTests();
+
+  // Returns the value of GTEST_FLAG(catch_exceptions) at the moment
+  // UnitTest::Run() starts.
+  bool catch_exceptions() const { return catch_exceptions_; }
+
+ private:
+  friend class ::testing::UnitTest;
+
+  // Used by UnitTest::Run() to capture the state of
+  // GTEST_FLAG(catch_exceptions) at the moment it starts.
+  void set_catch_exceptions(bool value) { catch_exceptions_ = value; }
+
+  // The UnitTest object that owns this implementation object.
+  UnitTest* const parent_;
+
+  // The working directory when the first TEST() or TEST_F() was
+  // executed.
+  internal::FilePath original_working_dir_;
+
+  // The default test part result reporters.
+  DefaultGlobalTestPartResultReporter default_global_test_part_result_reporter_;
+  DefaultPerThreadTestPartResultReporter
+      default_per_thread_test_part_result_reporter_;
+
+  // Points to (but doesn't own) the global test part result reporter.
+  TestPartResultReporterInterface* global_test_part_result_repoter_;
+
+  // Protects read and write access to global_test_part_result_reporter_.
+  internal::Mutex global_test_part_result_reporter_mutex_;
+
+  // Points to (but doesn't own) the per-thread test part result reporter.
+  internal::ThreadLocal<TestPartResultReporterInterface*>
+      per_thread_test_part_result_reporter_;
+
+  // The vector of environments that need to be set-up/torn-down
+  // before/after the tests are run.
+  std::vector<Environment*> environments_;
+
+  // The vector of TestCases in their original order.  It owns the
+  // elements in the vector.
+  std::vector<TestCase*> test_cases_;
+
+  // Provides a level of indirection for the test case list to allow
+  // easy shuffling and restoring the test case order.  The i-th
+  // element of this vector is the index of the i-th test case in the
+  // shuffled order.
+  std::vector<int> test_case_indices_;
+
+#if GTEST_HAS_PARAM_TEST
+  // ParameterizedTestRegistry object used to register value-parameterized
+  // tests.
+  internal::ParameterizedTestCaseRegistry parameterized_test_registry_;
+
+  // Indicates whether RegisterParameterizedTests() has been called already.
+  bool parameterized_tests_registered_;
+#endif  // GTEST_HAS_PARAM_TEST
+
+  // Index of the last death test case registered.  Initially -1.
+  int last_death_test_case_;
+
+  // This points to the TestCase for the currently running test.  It
+  // changes as Google Test goes through one test case after another.
+  // When no test is running, this is set to NULL and Google Test
+  // stores assertion results in ad_hoc_test_result_.  Initially NULL.
+  TestCase* current_test_case_;
+
+  // This points to the TestInfo for the currently running test.  It
+  // changes as Google Test goes through one test after another.  When
+  // no test is running, this is set to NULL and Google Test stores
+  // assertion results in ad_hoc_test_result_.  Initially NULL.
+  TestInfo* current_test_info_;
+
+  // Normally, a user only writes assertions inside a TEST or TEST_F,
+  // or inside a function called by a TEST or TEST_F.  Since Google
+  // Test keeps track of which test is current running, it can
+  // associate such an assertion with the test it belongs to.
+  //
+  // If an assertion is encountered when no TEST or TEST_F is running,
+  // Google Test attributes the assertion result to an imaginary "ad hoc"
+  // test, and records the result in ad_hoc_test_result_.
+  TestResult ad_hoc_test_result_;
+
+  // The list of event listeners that can be used to track events inside
+  // Google Test.
+  TestEventListeners listeners_;
+
+  // The OS stack trace getter.  Will be deleted when the UnitTest
+  // object is destructed.  By default, an OsStackTraceGetter is used,
+  // but the user can set this field to use a custom getter if that is
+  // desired.
+  OsStackTraceGetterInterface* os_stack_trace_getter_;
+
+  // True iff PostFlagParsingInit() has been called.
+  bool post_flag_parse_init_performed_;
+
+  // The random number seed used at the beginning of the test run.
+  int random_seed_;
+
+  // Our random number generator.
+  internal::Random random_;
+
+  // The time of the test program start, in ms from the start of the
+  // UNIX epoch.
+  TimeInMillis start_timestamp_;
+
+  // How long the test took to run, in milliseconds.
+  TimeInMillis elapsed_time_;
+
+#if GTEST_HAS_DEATH_TEST
+  // The decomposed components of the gtest_internal_run_death_test flag,
+  // parsed when RUN_ALL_TESTS is called.
+  internal::scoped_ptr<InternalRunDeathTestFlag> internal_run_death_test_flag_;
+  internal::scoped_ptr<internal::DeathTestFactory> death_test_factory_;
+#endif  // GTEST_HAS_DEATH_TEST
+
+  // A per-thread stack of traces created by the SCOPED_TRACE() macro.
+  internal::ThreadLocal<std::vector<TraceInfo> > gtest_trace_stack_;
+
+  // The value of GTEST_FLAG(catch_exceptions) at the moment RunAllTests()
+  // starts.
+  bool catch_exceptions_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(UnitTestImpl);
+};  // class UnitTestImpl
+
+// Convenience function for accessing the global UnitTest
+// implementation object.
+inline UnitTestImpl* GetUnitTestImpl() {
+  return UnitTest::GetInstance()->impl();
+}
+
+#if GTEST_USES_SIMPLE_RE
+
+// Internal helper functions for implementing the simple regular
+// expression matcher.
+GTEST_API_ bool IsInSet(char ch, const char* str);
+GTEST_API_ bool IsAsciiDigit(char ch);
+GTEST_API_ bool IsAsciiPunct(char ch);
+GTEST_API_ bool IsRepeat(char ch);
+GTEST_API_ bool IsAsciiWhiteSpace(char ch);
+GTEST_API_ bool IsAsciiWordChar(char ch);
+GTEST_API_ bool IsValidEscape(char ch);
+GTEST_API_ bool AtomMatchesChar(bool escaped, char pattern, char ch);
+GTEST_API_ bool ValidateRegex(const char* regex);
+GTEST_API_ bool MatchRegexAtHead(const char* regex, const char* str);
+GTEST_API_ bool MatchRepetitionAndRegexAtHead(
+    bool escaped, char ch, char repeat, const char* regex, const char* str);
+GTEST_API_ bool MatchRegexAnywhere(const char* regex, const char* str);
+
+#endif  // GTEST_USES_SIMPLE_RE
+
+// Parses the command line for Google Test flags, without initializing
+// other parts of Google Test.
+GTEST_API_ void ParseGoogleTestFlagsOnly(int* argc, char** argv);
+GTEST_API_ void ParseGoogleTestFlagsOnly(int* argc, wchar_t** argv);
+
+#if GTEST_HAS_DEATH_TEST
+
+// Returns the message describing the last system error, regardless of the
+// platform.
+GTEST_API_ std::string GetLastErrnoDescription();
+
+// Attempts to parse a string into a positive integer pointed to by the
+// number parameter.  Returns true if that is possible.
+// GTEST_HAS_DEATH_TEST implies that we have ::std::string, so we can use
+// it here.
+template <typename Integer>
+bool ParseNaturalNumber(const ::std::string& str, Integer* number) {
+  // Fail fast if the given string does not begin with a digit;
+  // this bypasses strtoXXX's "optional leading whitespace and plus
+  // or minus sign" semantics, which are undesirable here.
+  if (str.empty() || !IsDigit(str[0])) {
+    return false;
+  }
+  errno = 0;
+
+  char* end;
+  // BiggestConvertible is the largest integer type that system-provided
+  // string-to-number conversion routines can return.
+
+# if GTEST_OS_WINDOWS && !defined(__GNUC__)
+
+  // MSVC and C++ Builder define __int64 instead of the standard long long.
+  typedef unsigned __int64 BiggestConvertible;
+  const BiggestConvertible parsed = _strtoui64(str.c_str(), &end, 10);
+
+# else
+
+  typedef unsigned long long BiggestConvertible;  // NOLINT
+  const BiggestConvertible parsed = strtoull(str.c_str(), &end, 10);
+
+# endif  // GTEST_OS_WINDOWS && !defined(__GNUC__)
+
+  const bool parse_success = *end == '\0' && errno == 0;
+
+  // TODO(vladl@google.com): Convert this to compile time assertion when it is
+  // available.
+  GTEST_CHECK_(sizeof(Integer) <= sizeof(parsed));
+
+  const Integer result = static_cast<Integer>(parsed);
+  if (parse_success && static_cast<BiggestConvertible>(result) == parsed) {
+    *number = result;
+    return true;
+  }
+  return false;
+}
+#endif  // GTEST_HAS_DEATH_TEST
+
+// TestResult contains some private methods that should be hidden from
+// Google Test user but are required for testing. This class allow our tests
+// to access them.
+//
+// This class is supplied only for the purpose of testing Google Test's own
+// constructs. Do not use it in user tests, either directly or indirectly.
+class TestResultAccessor {
+ public:
+  static void RecordProperty(TestResult* test_result,
+                             const std::string& xml_element,
+                             const TestProperty& property) {
+    test_result->RecordProperty(xml_element, property);
+  }
+
+  static void ClearTestPartResults(TestResult* test_result) {
+    test_result->ClearTestPartResults();
+  }
+
+  static const std::vector<testing::TestPartResult>& test_part_results(
+      const TestResult& test_result) {
+    return test_result.test_part_results();
+  }
+};
+
+#if GTEST_CAN_STREAM_RESULTS_
+
+// Streams test results to the given port on the given host machine.
+class GTEST_API_ StreamingListener : public EmptyTestEventListener {
+ public:
+  // Abstract base class for writing strings to a socket.
+  class AbstractSocketWriter {
+   public:
+    virtual ~AbstractSocketWriter() {}
+
+    // Sends a string to the socket.
+    virtual void Send(const string& message) = 0;
+
+    // Closes the socket.
+    virtual void CloseConnection() {}
+
+    // Sends a string and a newline to the socket.
+    void SendLn(const string& message) {
+      Send(message + "\n");
+    }
+  };
+
+  // Concrete class for actually writing strings to a socket.
+  class SocketWriter : public AbstractSocketWriter {
+   public:
+    SocketWriter(const string& host, const string& port)
+        : sockfd_(-1), host_name_(host), port_num_(port) {
+      MakeConnection();
+    }
+
+    virtual ~SocketWriter() {
+      if (sockfd_ != -1)
+        CloseConnection();
+    }
+
+    // Sends a string to the socket.
+    virtual void Send(const string& message) {
+      GTEST_CHECK_(sockfd_ != -1)
+          << "Send() can be called only when there is a connection.";
+
+      const int len = static_cast<int>(message.length());
+      if (write(sockfd_, message.c_str(), len) != len) {
+        GTEST_LOG_(WARNING)
+            << "stream_result_to: failed to stream to "
+            << host_name_ << ":" << port_num_;
+      }
+    }
+
+   private:
+    // Creates a client socket and connects to the server.
+    void MakeConnection();
+
+    // Closes the socket.
+    void CloseConnection() {
+      GTEST_CHECK_(sockfd_ != -1)
+          << "CloseConnection() can be called only when there is a connection.";
+
+      close(sockfd_);
+      sockfd_ = -1;
+    }
+
+    int sockfd_;  // socket file descriptor
+    const string host_name_;
+    const string port_num_;
+
+    GTEST_DISALLOW_COPY_AND_ASSIGN_(SocketWriter);
+  };  // class SocketWriter
+
+  // Escapes '=', '&', '%', and '\n' characters in str as "%xx".
+  static string UrlEncode(const char* str);
+
+  StreamingListener(const string& host, const string& port)
+      : socket_writer_(new SocketWriter(host, port)) { Start(); }
+
+  explicit StreamingListener(AbstractSocketWriter* socket_writer)
+      : socket_writer_(socket_writer) { Start(); }
+
+  void OnTestProgramStart(const UnitTest& /* unit_test */) {
+    SendLn("event=TestProgramStart");
+  }
+
+  void OnTestProgramEnd(const UnitTest& unit_test) {
+    // Note that Google Test current only report elapsed time for each
+    // test iteration, not for the entire test program.
+    SendLn("event=TestProgramEnd&passed=" + FormatBool(unit_test.Passed()));
+
+    // Notify the streaming server to stop.
+    socket_writer_->CloseConnection();
+  }
+
+  void OnTestIterationStart(const UnitTest& /* unit_test */, int iteration) {
+    SendLn("event=TestIterationStart&iteration=" +
+           StreamableToString(iteration));
+  }
+
+  void OnTestIterationEnd(const UnitTest& unit_test, int /* iteration */) {
+    SendLn("event=TestIterationEnd&passed=" +
+           FormatBool(unit_test.Passed()) + "&elapsed_time=" +
+           StreamableToString(unit_test.elapsed_time()) + "ms");
+  }
+
+  void OnTestCaseStart(const TestCase& test_case) {
+    SendLn(std::string("event=TestCaseStart&name=") + test_case.name());
+  }
+
+  void OnTestCaseEnd(const TestCase& test_case) {
+    SendLn("event=TestCaseEnd&passed=" + FormatBool(test_case.Passed())
+           + "&elapsed_time=" + StreamableToString(test_case.elapsed_time())
+           + "ms");
+  }
+
+  void OnTestStart(const TestInfo& test_info) {
+    SendLn(std::string("event=TestStart&name=") + test_info.name());
+  }
+
+  void OnTestEnd(const TestInfo& test_info) {
+    SendLn("event=TestEnd&passed=" +
+           FormatBool((test_info.result())->Passed()) +
+           "&elapsed_time=" +
+           StreamableToString((test_info.result())->elapsed_time()) + "ms");
+  }
+
+  void OnTestPartResult(const TestPartResult& test_part_result) {
+    const char* file_name = test_part_result.file_name();
+    if (file_name == NULL)
+      file_name = "";
+    SendLn("event=TestPartResult&file=" + UrlEncode(file_name) +
+           "&line=" + StreamableToString(test_part_result.line_number()) +
+           "&message=" + UrlEncode(test_part_result.message()));
+  }
+
+ private:
+  // Sends the given message and a newline to the socket.
+  void SendLn(const string& message) { socket_writer_->SendLn(message); }
+
+  // Called at the start of streaming to notify the receiver what
+  // protocol we are using.
+  void Start() { SendLn("gtest_streaming_protocol_version=1.0"); }
+
+  string FormatBool(bool value) { return value ? "1" : "0"; }
+
+  const scoped_ptr<AbstractSocketWriter> socket_writer_;
+
+  GTEST_DISALLOW_COPY_AND_ASSIGN_(StreamingListener);
+};  // class StreamingListener
+
+#endif  // GTEST_CAN_STREAM_RESULTS_
+
+}  // namespace internal
+}  // namespace testing
+
+#endif  // GTEST_SRC_GTEST_INTERNAL_INL_H_
+#undef GTEST_IMPLEMENTATION_
+
+#if GTEST_OS_WINDOWS
+# define vsnprintf _vsnprintf
+#endif  // GTEST_OS_WINDOWS
+
+namespace testing {
+
+using internal::CountIf;
+using internal::ForEach;
+using internal::GetElementOr;
+using internal::Shuffle;
+
+// Constants.
+
+// A test whose test case name or test name matches this filter is
+// disabled and not run.
+static const char kDisableTestFilter[] = "DISABLED_*:*/DISABLED_*";
+
+// A test case whose name matches this filter is considered a death
+// test case and will be run before test cases whose name doesn't
+// match this filter.
+static const char kDeathTestCaseFilter[] = "*DeathTest:*DeathTest/*";
+
+// A test filter that matches everything.
+static const char kUniversalFilter[] = "*";
+
+// The default output file for XML output.
+static const char kDefaultOutputFile[] = "test_detail.xml";
+
+// The environment variable name for the test shard index.
+static const char kTestShardIndex[] = "GTEST_SHARD_INDEX";
+// The environment variable name for the total number of test shards.
+static const char kTestTotalShards[] = "GTEST_TOTAL_SHARDS";
+// The environment variable name for the test shard status file.
+static const char kTestShardStatusFile[] = "GTEST_SHARD_STATUS_FILE";
+
+namespace internal {
+
+// The text used in failure messages to indicate the start of the
+// stack trace.
+const char kStackTraceMarker[] = "\nStack trace:\n";
+
+// g_help_flag is true iff the --help flag or an equivalent form is
+// specified on the command line.
+bool g_help_flag = false;
+
+}  // namespace internal
+
+static const char* GetDefaultFilter() {
+#ifdef GTEST_TEST_FILTER_ENV_VAR_
+  const char* const testbridge_test_only = getenv(GTEST_TEST_FILTER_ENV_VAR_);
+  if (testbridge_test_only != NULL) {
+    return testbridge_test_only;
+  }
+#endif  // GTEST_TEST_FILTER_ENV_VAR_
+  return kUniversalFilter;
+}
+
+GTEST_DEFINE_bool_(
+    also_run_disabled_tests,
+    internal::BoolFromGTestEnv("also_run_disabled_tests", false),
+    "Run disabled tests too, in addition to the tests normally being run.");
+
+GTEST_DEFINE_bool_(
+    break_on_failure,
+    internal::BoolFromGTestEnv("break_on_failure", false),
+    "True iff a failed assertion should be a debugger break-point.");
+
+GTEST_DEFINE_bool_(
+    catch_exceptions,
+    internal::BoolFromGTestEnv("catch_exceptions", true),
+    "True iff " GTEST_NAME_
+    " should catch exceptions and treat them as test failures.");
+
+GTEST_DEFINE_string_(
+    color,
+    internal::StringFromGTestEnv("color", "auto"),
+    "Whether to use colors in the output.  Valid values: yes, no, "
+    "and auto.  'auto' means to use colors if the output is "
+    "being sent to a terminal and the TERM environment variable "
+    "is set to a terminal type that supports colors.");
+
+GTEST_DEFINE_string_(
+    filter,
+    internal::StringFromGTestEnv("filter", GetDefaultFilter()),
+    "A colon-separated list of glob (not regex) patterns "
+    "for filtering the tests to run, optionally followed by a "
+    "'-' and a : separated list of negative patterns (tests to "
+    "exclude).  A test is run if it matches one of the positive "
+    "patterns and does not match any of the negative patterns.");
+
+GTEST_DEFINE_bool_(list_tests, false,
+                   "List all tests without running them.");
+
+GTEST_DEFINE_string_(
+    output,
+    internal::StringFromGTestEnv("output", ""),
+    "A format (currently must be \"xml\"), optionally followed "
+    "by a colon and an output file name or directory. A directory "
+    "is indicated by a trailing pathname separator. "
+    "Examples: \"xml:filename.xml\", \"xml::directoryname/\". "
+    "If a directory is specified, output files will be created "
+    "within that directory, with file-names based on the test "
+    "executable's name and, if necessary, made unique by adding "
+    "digits.");
+
+GTEST_DEFINE_bool_(
+    print_time,
+    internal::BoolFromGTestEnv("print_time", true),
+    "True iff " GTEST_NAME_
+    " should display elapsed time in text output.");
+
+GTEST_DEFINE_int32_(
+    random_seed,
+    internal::Int32FromGTestEnv("random_seed", 0),
+    "Random number seed to use when shuffling test orders.  Must be in range "
+    "[1, 99999], or 0 to use a seed based on the current time.");
+
+GTEST_DEFINE_int32_(
+    repeat,
+    internal::Int32FromGTestEnv("repeat", 1),
+    "How many times to repeat each test.  Specify a negative number "
+    "for repeating forever.  Useful for shaking out flaky tests.");
+
+GTEST_DEFINE_bool_(
+    show_internal_stack_frames, false,
+    "True iff " GTEST_NAME_ " should include internal stack frames when "
+    "printing test failure stack traces.");
+
+GTEST_DEFINE_bool_(
+    shuffle,
+    internal::BoolFromGTestEnv("shuffle", false),
+    "True iff " GTEST_NAME_
+    " should randomize tests' order on every run.");
+
+GTEST_DEFINE_int32_(
+    stack_trace_depth,
+    internal::Int32FromGTestEnv("stack_trace_depth", kMaxStackTraceDepth),
+    "The maximum number of stack frames to print when an "
+    "assertion fails.  The valid range is 0 through 100, inclusive.");
+
+GTEST_DEFINE_string_(
+    stream_result_to,
+    internal::StringFromGTestEnv("stream_result_to", ""),
+    "This flag specifies the host name and the port number on which to stream "
+    "test results. Example: \"localhost:555\". The flag is effective only on "
+    "Linux.");
+
+GTEST_DEFINE_bool_(
+    throw_on_failure,
+    internal::BoolFromGTestEnv("throw_on_failure", false),
+    "When this flag is specified, a failed assertion will throw an exception "
+    "if exceptions are enabled or exit the program with a non-zero code "
+    "otherwise.");
+
+#if GTEST_USE_OWN_FLAGFILE_FLAG_
+GTEST_DEFINE_string_(
+    flagfile,
+    internal::StringFromGTestEnv("flagfile", ""),
+    "This flag specifies the flagfile to read command-line flags from.");
+#endif  // GTEST_USE_OWN_FLAGFILE_FLAG_
+
+namespace internal {
+
+// Generates a random number from [0, range), using a Linear
+// Congruential Generator (LCG).  Crashes if 'range' is 0 or greater
+// than kMaxRange.
+UInt32 Random::Generate(UInt32 range) {
+  // These constants are the same as are used in glibc's rand(3).
+  state_ = (1103515245U*state_ + 12345U) % kMaxRange;
+
+  GTEST_CHECK_(range > 0)
+      << "Cannot generate a number in the range [0, 0).";
+  GTEST_CHECK_(range <= kMaxRange)
+      << "Generation of a number in [0, " << range << ") was requested, "
+      << "but this can only generate numbers in [0, " << kMaxRange << ").";
+
+  // Converting via modulus introduces a bit of downward bias, but
+  // it's simple, and a linear congruential generator isn't too good
+  // to begin with.
+  return state_ % range;
+}
+
+// GTestIsInitialized() returns true iff the user has initialized
+// Google Test.  Useful for catching the user mistake of not initializing
+// Google Test before calling RUN_ALL_TESTS().
+static bool GTestIsInitialized() { return GetArgvs().size() > 0; }
+
+// Iterates over a vector of TestCases, keeping a running sum of the
+// results of calling a given int-returning method on each.
+// Returns the sum.
+static int SumOverTestCaseList(const std::vector<TestCase*>& case_list,
+                               int (TestCase::*method)() const) {
+  int sum = 0;
+  for (size_t i = 0; i < case_list.size(); i++) {
+    sum += (case_list[i]->*method)();
+  }
+  return sum;
+}
+
+// Returns true iff the test case passed.
+static bool TestCasePassed(const TestCase* test_case) {
+  return test_case->should_run() && test_case->Passed();
+}
+
+// Returns true iff the test case failed.
+static bool TestCaseFailed(const TestCase* test_case) {
+  return test_case->should_run() && test_case->Failed();
+}
+
+// Returns true iff test_case contains at least one test that should
+// run.
+static bool ShouldRunTestCase(const TestCase* test_case) {
+  return test_case->should_run();
+}
+
+// AssertHelper constructor.
+AssertHelper::AssertHelper(TestPartResult::Type type,
+                           const char* file,
+                           int line,
+                           const char* message)
+    : data_(new AssertHelperData(type, file, line, message)) {
+}
+
+AssertHelper::~AssertHelper() {
+  delete data_;
+}
+
+// Message assignment, for assertion streaming support.
+void AssertHelper::operator=(const Message& message) const {
+  UnitTest::GetInstance()->
+    AddTestPartResult(data_->type, data_->file, data_->line,
+                      AppendUserMessage(data_->message, message),
+                      UnitTest::GetInstance()->impl()
+                      ->CurrentOsStackTraceExceptTop(1)
+                      // Skips the stack frame for this function itself.
+                      );  // NOLINT
+}
+
+// Mutex for linked pointers.
+GTEST_API_ GTEST_DEFINE_STATIC_MUTEX_(g_linked_ptr_mutex);
+
+// A copy of all command line arguments.  Set by InitGoogleTest().
+::std::vector<testing::internal::string> g_argvs;
+
+const ::std::vector<testing::internal::string>& GetArgvs() {
+#if defined(GTEST_CUSTOM_GET_ARGVS_)
+  return GTEST_CUSTOM_GET_ARGVS_();
+#else  // defined(GTEST_CUSTOM_GET_ARGVS_)
+  return g_argvs;
+#endif  // defined(GTEST_CUSTOM_GET_ARGVS_)
+}
+
+// Returns the current application's name, removing directory path if that
+// is present.
+FilePath GetCurrentExecutableName() {
+  FilePath result;
+
+#if GTEST_OS_WINDOWS
+  result.Set(FilePath(GetArgvs()[0]).RemoveExtension("exe"));
+#else
+  result.Set(FilePath(GetArgvs()[0]));
+#endif  // GTEST_OS_WINDOWS
+
+  return result.RemoveDirectoryName();
+}
+
+// Functions for processing the gtest_output flag.
+
+// Returns the output format, or "" for normal printed output.
+std::string UnitTestOptions::GetOutputFormat() {
+  const char* const gtest_output_flag = GTEST_FLAG(output).c_str();
+  if (gtest_output_flag == NULL) return std::string("");
+
+  const char* const colon = strchr(gtest_output_flag, ':');
+  return (colon == NULL) ?
+      std::string(gtest_output_flag) :
+      std::string(gtest_output_flag, colon - gtest_output_flag);
+}
+
+// Returns the name of the requested output file, or the default if none
+// was explicitly specified.
+std::string UnitTestOptions::GetAbsolutePathToOutputFile() {
+  const char* const gtest_output_flag = GTEST_FLAG(output).c_str();
+  if (gtest_output_flag == NULL)
+    return "";
+
+  const char* const colon = strchr(gtest_output_flag, ':');
+  if (colon == NULL)
+    return internal::FilePath::ConcatPaths(
+        internal::FilePath(
+            UnitTest::GetInstance()->original_working_dir()),
+        internal::FilePath(kDefaultOutputFile)).string();
+
+  internal::FilePath output_name(colon + 1);
+  if (!output_name.IsAbsolutePath())
+    // TODO(wan@google.com): on Windows \some\path is not an absolute
+    // path (as its meaning depends on the current drive), yet the
+    // following logic for turning it into an absolute path is wrong.
+    // Fix it.
+    output_name = internal::FilePath::ConcatPaths(
+        internal::FilePath(UnitTest::GetInstance()->original_working_dir()),
+        internal::FilePath(colon + 1));
+
+  if (!output_name.IsDirectory())
+    return output_name.string();
+
+  internal::FilePath result(internal::FilePath::GenerateUniqueFileName(
+      output_name, internal::GetCurrentExecutableName(),
+      GetOutputFormat().c_str()));
+  return result.string();
+}
+
+// Returns true iff the wildcard pattern matches the string.  The
+// first ':' or '\0' character in pattern marks the end of it.
+//
+// This recursive algorithm isn't very efficient, but is clear and
+// works well enough for matching test names, which are short.
+bool UnitTestOptions::PatternMatchesString(const char *pattern,
+                                           const char *str) {
+  switch (*pattern) {
+    case '\0':
+    case ':':  // Either ':' or '\0' marks the end of the pattern.
+      return *str == '\0';
+    case '?':  // Matches any single character.
+      return *str != '\0' && PatternMatchesString(pattern + 1, str + 1);
+    case '*':  // Matches any string (possibly empty) of characters.
+      return (*str != '\0' && PatternMatchesString(pattern, str + 1)) ||
+          PatternMatchesString(pattern + 1, str);
+    default:  // Non-special character.  Matches itself.
+      return *pattern == *str &&
+          PatternMatchesString(pattern + 1, str + 1);
+  }
+}
+
+bool UnitTestOptions::MatchesFilter(
+    const std::string& name, const char* filter) {
+  const char *cur_pattern = filter;
+  for (;;) {
+    if (PatternMatchesString(cur_pattern, name.c_str())) {
+      return true;
+    }
+
+    // Finds the next pattern in the filter.
+    cur_pattern = strchr(cur_pattern, ':');
+
+    // Returns if no more pattern can be found.
+    if (cur_pattern == NULL) {
+      return false;
+    }
+
+    // Skips the pattern separater (the ':' character).
+    cur_pattern++;
+  }
+}
+
+// Returns true iff the user-specified filter matches the test case
+// name and the test name.
+bool UnitTestOptions::FilterMatchesTest(const std::string &test_case_name,
+                                        const std::string &test_name) {
+  const std::string& full_name = test_case_name + "." + test_name.c_str();
+
+  // Split --gtest_filter at '-', if there is one, to separate into
+  // positive filter and negative filter portions
+  const char* const p = GTEST_FLAG(filter).c_str();
+  const char* const dash = strchr(p, '-');
+  std::string positive;
+  std::string negative;
+  if (dash == NULL) {
+    positive = GTEST_FLAG(filter).c_str();  // Whole string is a positive filter
+    negative = "";
+  } else {
+    positive = std::string(p, dash);   // Everything up to the dash
+    negative = std::string(dash + 1);  // Everything after the dash
+    if (positive.empty()) {
+      // Treat '-test1' as the same as '*-test1'
+      positive = kUniversalFilter;
+    }
+  }
+
+  // A filter is a colon-separated list of patterns.  It matches a
+  // test if any pattern in it matches the test.
+  return (MatchesFilter(full_name, positive.c_str()) &&
+          !MatchesFilter(full_name, negative.c_str()));
+}
+
+#if GTEST_HAS_SEH
+// Returns EXCEPTION_EXECUTE_HANDLER if Google Test should handle the
+// given SEH exception, or EXCEPTION_CONTINUE_SEARCH otherwise.
+// This function is useful as an __except condition.
+int UnitTestOptions::GTestShouldProcessSEH(DWORD exception_code) {
+  // Google Test should handle a SEH exception if:
+  //   1. the user wants it to, AND
+  //   2. this is not a breakpoint exception, AND
+  //   3. this is not a C++ exception (VC++ implements them via SEH,
+  //      apparently).
+  //
+  // SEH exception code for C++ exceptions.
+  // (see http://support.microsoft.com/kb/185294 for more information).
+  const DWORD kCxxExceptionCode = 0xe06d7363;
+
+  bool should_handle = true;
+
+  if (!GTEST_FLAG(catch_exceptions))
+    should_handle = false;
+  else if (exception_code == EXCEPTION_BREAKPOINT)
+    should_handle = false;
+  else if (exception_code == kCxxExceptionCode)
+    should_handle = false;
+
+  return should_handle ? EXCEPTION_EXECUTE_HANDLER : EXCEPTION_CONTINUE_SEARCH;
+}
+#endif  // GTEST_HAS_SEH
+
+}  // namespace internal
+
+// The c'tor sets this object as the test part result reporter used by
+// Google Test.  The 'result' parameter specifies where to report the
+// results. Intercepts only failures from the current thread.
+ScopedFakeTestPartResultReporter::ScopedFakeTestPartResultReporter(
+    TestPartResultArray* result)
+    : intercept_mode_(INTERCEPT_ONLY_CURRENT_THREAD),
+      result_(result) {
+  Init();
+}
+
+// The c'tor sets this object as the test part result reporter used by
+// Google Test.  The 'result' parameter specifies where to report the
+// results.
+ScopedFakeTestPartResultReporter::ScopedFakeTestPartResultReporter(
+    InterceptMode intercept_mode, TestPartResultArray* result)
+    : intercept_mode_(intercept_mode),
+      result_(result) {
+  Init();
+}
+
+void ScopedFakeTestPartResultReporter::Init() {
+  internal::UnitTestImpl* const impl = internal::GetUnitTestImpl();
+  if (intercept_mode_ == INTERCEPT_ALL_THREADS) {
+    old_reporter_ = impl->GetGlobalTestPartResultReporter();
+    impl->SetGlobalTestPartResultReporter(this);
+  } else {
+    old_reporter_ = impl->GetTestPartResultReporterForCurrentThread();
+    impl->SetTestPartResultReporterForCurrentThread(this);
+  }
+}
+
+// The d'tor restores the test part result reporter used by Google Test
+// before.
+ScopedFakeTestPartResultReporter::~ScopedFakeTestPartResultReporter() {
+  internal::UnitTestImpl* const impl = internal::GetUnitTestImpl();
+  if (intercept_mode_ == INTERCEPT_ALL_THREADS) {
+    impl->SetGlobalTestPartResultReporter(old_reporter_);
+  } else {
+    impl->SetTestPartResultReporterForCurrentThread(old_reporter_);
+  }
+}
+
+// Increments the test part result count and remembers the result.
+// This method is from the TestPartResultReporterInterface interface.
+void ScopedFakeTestPartResultReporter::ReportTestPartResult(
+    const TestPartResult& result) {
+  result_->Append(result);
+}
+
+namespace internal {
+
+// Returns the type ID of ::testing::Test.  We should always call this
+// instead of GetTypeId< ::testing::Test>() to get the type ID of
+// testing::Test.  This is to work around a suspected linker bug when
+// using Google Test as a framework on Mac OS X.  The bug causes
+// GetTypeId< ::testing::Test>() to return different values depending
+// on whether the call is from the Google Test framework itself or
+// from user test code.  GetTestTypeId() is guaranteed to always
+// return the same value, as it always calls GetTypeId<>() from the
+// gtest.cc, which is within the Google Test framework.
+TypeId GetTestTypeId() {
+  return GetTypeId<Test>();
+}
+
+// The value of GetTestTypeId() as seen from within the Google Test
+// library.  This is solely for testing GetTestTypeId().
+extern const TypeId kTestTypeIdInGoogleTest = GetTestTypeId();
+
+// This predicate-formatter checks that 'results' contains a test part
+// failure of the given type and that the failure message contains the
+// given substring.
+AssertionResult HasOneFailure(const char* /* results_expr */,
+                              const char* /* type_expr */,
+                              const char* /* substr_expr */,
+                              const TestPartResultArray& results,
+                              TestPartResult::Type type,
+                              const string& substr) {
+  const std::string expected(type == TestPartResult::kFatalFailure ?
+                        "1 fatal failure" :
+                        "1 non-fatal failure");
+  Message msg;
+  if (results.size() != 1) {
+    msg << "Expected: " << expected << "\n"
+        << "  Actual: " << results.size() << " failures";
+    for (int i = 0; i < results.size(); i++) {
+      msg << "\n" << results.GetTestPartResult(i);
+    }
+    return AssertionFailure() << msg;
+  }
+
+  const TestPartResult& r = results.GetTestPartResult(0);
+  if (r.type() != type) {
+    return AssertionFailure() << "Expected: " << expected << "\n"
+                              << "  Actual:\n"
+                              << r;
+  }
+
+  if (strstr(r.message(), substr.c_str()) == NULL) {
+    return AssertionFailure() << "Expected: " << expected << " containing \""
+                              << substr << "\"\n"
+                              << "  Actual:\n"
+                              << r;
+  }
+
+  return AssertionSuccess();
+}
+
+// The constructor of SingleFailureChecker remembers where to look up
+// test part results, what type of failure we expect, and what
+// substring the failure message should contain.
+SingleFailureChecker:: SingleFailureChecker(
+    const TestPartResultArray* results,
+    TestPartResult::Type type,
+    const string& substr)
+    : results_(results),
+      type_(type),
+      substr_(substr) {}
+
+// The destructor of SingleFailureChecker verifies that the given
+// TestPartResultArray contains exactly one failure that has the given
+// type and contains the given substring.  If that's not the case, a
+// non-fatal failure will be generated.
+SingleFailureChecker::~SingleFailureChecker() {
+  EXPECT_PRED_FORMAT3(HasOneFailure, *results_, type_, substr_);
+}
+
+DefaultGlobalTestPartResultReporter::DefaultGlobalTestPartResultReporter(
+    UnitTestImpl* unit_test) : unit_test_(unit_test) {}
+
+void DefaultGlobalTestPartResultReporter::ReportTestPartResult(
+    const TestPartResult& result) {
+  unit_test_->current_test_result()->AddTestPartResult(result);
+  unit_test_->listeners()->repeater()->OnTestPartResult(result);
+}
+
+DefaultPerThreadTestPartResultReporter::DefaultPerThreadTestPartResultReporter(
+    UnitTestImpl* unit_test) : unit_test_(unit_test) {}
+
+void DefaultPerThreadTestPartResultReporter::ReportTestPartResult(
+    const TestPartResult& result) {
+  unit_test_->GetGlobalTestPartResultReporter()->ReportTestPartResult(result);
+}
+
+// Returns the global test part result reporter.
+TestPartResultReporterInterface*
+UnitTestImpl::GetGlobalTestPartResultReporter() {
+  internal::MutexLock lock(&global_test_part_result_reporter_mutex_);
+  return global_test_part_result_repoter_;
+}
+
+// Sets the global test part result reporter.
+void UnitTestImpl::SetGlobalTestPartResultReporter(
+    TestPartResultReporterInterface* reporter) {
+  internal::MutexLock lock(&global_test_part_result_reporter_mutex_);
+  global_test_part_result_repoter_ = reporter;
+}
+
+// Returns the test part result reporter for the current thread.
+TestPartResultReporterInterface*
+UnitTestImpl::GetTestPartResultReporterForCurrentThread() {
+  return per_thread_test_part_result_reporter_.get();
+}
+
+// Sets the test part result reporter for the current thread.
+void UnitTestImpl::SetTestPartResultReporterForCurrentThread(
+    TestPartResultReporterInterface* reporter) {
+  per_thread_test_part_result_reporter_.set(reporter);
+}
+
+// Gets the number of successful test cases.
+int UnitTestImpl::successful_test_case_count() const {
+  return CountIf(test_cases_, TestCasePassed);
+}
+
+// Gets the number of failed test cases.
+int UnitTestImpl::failed_test_case_count() const {
+  return CountIf(test_cases_, TestCaseFailed);
+}
+
+// Gets the number of all test cases.
+int UnitTestImpl::total_test_case_count() const {
+  return static_cast<int>(test_cases_.size());
+}
+
+// Gets the number of all test cases that contain at least one test
+// that should run.
+int UnitTestImpl::test_case_to_run_count() const {
+  return CountIf(test_cases_, ShouldRunTestCase);
+}
+
+// Gets the number of successful tests.
+int UnitTestImpl::successful_test_count() const {
+  return SumOverTestCaseList(test_cases_, &TestCase::successful_test_count);
+}
+
+// Gets the number of failed tests.
+int UnitTestImpl::failed_test_count() const {
+  return SumOverTestCaseList(test_cases_, &TestCase::failed_test_count);
+}
+
+// Gets the number of disabled tests that will be reported in the XML report.
+int UnitTestImpl::reportable_disabled_test_count() const {
+  return SumOverTestCaseList(test_cases_,
+                             &TestCase::reportable_disabled_test_count);
+}
+
+// Gets the number of disabled tests.
+int UnitTestImpl::disabled_test_count() const {
+  return SumOverTestCaseList(test_cases_, &TestCase::disabled_test_count);
+}
+
+// Gets the number of tests to be printed in the XML report.
+int UnitTestImpl::reportable_test_count() const {
+  return SumOverTestCaseList(test_cases_, &TestCase::reportable_test_count);
+}
+
+// Gets the number of all tests.
+int UnitTestImpl::total_test_count() const {
+  return SumOverTestCaseList(test_cases_, &TestCase::total_test_count);
+}
+
+// Gets the number of tests that should run.
+int UnitTestImpl::test_to_run_count() const {
+  return SumOverTestCaseList(test_cases_, &TestCase::test_to_run_count);
+}
+
+// Returns the current OS stack trace as an std::string.
+//
+// The maximum number of stack frames to be included is specified by
+// the gtest_stack_trace_depth flag.  The skip_count parameter
+// specifies the number of top frames to be skipped, which doesn't
+// count against the number of frames to be included.
+//
+// For example, if Foo() calls Bar(), which in turn calls
+// CurrentOsStackTraceExceptTop(1), Foo() will be included in the
+// trace but Bar() and CurrentOsStackTraceExceptTop() won't.
+std::string UnitTestImpl::CurrentOsStackTraceExceptTop(int skip_count) {
+  return os_stack_trace_getter()->CurrentStackTrace(
+      static_cast<int>(GTEST_FLAG(stack_trace_depth)),
+      skip_count + 1
+      // Skips the user-specified number of frames plus this function
+      // itself.
+      );  // NOLINT
+}
+
+// Returns the current time in milliseconds.
+TimeInMillis GetTimeInMillis() {
+#if GTEST_OS_WINDOWS_MOBILE || defined(__BORLANDC__)
+  // Difference between 1970-01-01 and 1601-01-01 in milliseconds.
+  // http://analogous.blogspot.com/2005/04/epoch.html
+  const TimeInMillis kJavaEpochToWinFileTimeDelta =
+    static_cast<TimeInMillis>(116444736UL) * 100000UL;
+  const DWORD kTenthMicrosInMilliSecond = 10000;
+
+  SYSTEMTIME now_systime;
+  FILETIME now_filetime;
+  ULARGE_INTEGER now_int64;
+  // TODO(kenton@google.com): Shouldn't this just use
+  //   GetSystemTimeAsFileTime()?
+  GetSystemTime(&now_systime);
+  if (SystemTimeToFileTime(&now_systime, &now_filetime)) {
+    now_int64.LowPart = now_filetime.dwLowDateTime;
+    now_int64.HighPart = now_filetime.dwHighDateTime;
+    now_int64.QuadPart = (now_in

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/50] [abbrv] hadoop git commit: HDFS-10581. Hide redundant table on NameNode WebUI when no nodes are decomissioning. Contributed by Weiwei Yang.

Posted by st...@apache.org.
HDFS-10581. Hide redundant table on NameNode WebUI when no nodes are decomissioning. Contributed by Weiwei Yang.


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

Branch: refs/heads/HADOOP-13345
Commit: 8e63fa98eabac55bdb2254306584ad1e759c79eb
Parents: a2b5d60
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Dec 5 18:13:53 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Dec 5 18:13:53 2016 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html             | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e63fa98/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
index b0db3a1..13569fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
@@ -352,6 +352,7 @@
 
 <div class="page-header"><h1><small>Decommissioning</small></h1></div>
 <small>
+{?DecomNodes}
 <table class="table">
   <thead>
     <tr>
@@ -370,6 +371,9 @@
   </tr>
   {/DecomNodes}
 </table>
+{:else}
+No nodes are decommissioning
+{/DecomNodes}
 </small>
 </script>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/50] [abbrv] hadoop git commit: HADOOP-13855. Fix a couple of the s3a statistic names to be consistent with the rest. Contributed by Steve Loughran

Posted by st...@apache.org.
HADOOP-13855. Fix a couple of the s3a statistic names to be consistent with the rest. Contributed by Steve Loughran


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

Branch: refs/heads/HADOOP-13345
Commit: 51211a7d7aa342b93951fe61da3f624f0652e101
Parents: 5bd18c4
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 2 13:48:15 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 2 14:01:42 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/fs/s3a/Statistic.java        | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51211a7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index 36ec50b..789c6d7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -92,12 +92,12 @@ public enum Statistic {
       "Count of times the TCP stream was aborted"),
   STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
       "Number of executed seek operations which went backwards in a stream"),
-  STREAM_CLOSED("streamClosed", "Count of times the TCP stream was closed"),
+  STREAM_CLOSED("stream_closed", "Count of times the TCP stream was closed"),
   STREAM_CLOSE_OPERATIONS("stream_close_operations",
       "Total count of times an attempt to close a data stream was made"),
   STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations",
       "Number of executed seek operations which went forward in a stream"),
-  STREAM_OPENED("streamOpened",
+  STREAM_OPENED("stream_opened",
       "Total count of times an input stream to object store was opened"),
   STREAM_READ_EXCEPTIONS("stream_read_exceptions",
       "Number of seek operations invoked on input streams"),


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/50] [abbrv] hadoop git commit: HADOOP-13864. KMS should not require truststore password. Contributed by Mike Yoder.

Posted by st...@apache.org.
HADOOP-13864. KMS should not require truststore password. Contributed by Mike Yoder.


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

Branch: refs/heads/HADOOP-13345
Commit: a2b5d602201a4f619f6a68ec2168a884190d8de6
Parents: f3b8ff5
Author: Xiao Chen <xi...@apache.org>
Authored: Mon Dec 5 12:19:26 2016 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Mon Dec 5 17:36:00 2016 -0800

----------------------------------------------------------------------
 .../security/ssl/FileBasedKeyStoresFactory.java   |  6 ++++--
 .../security/ssl/ReloadingX509TrustManager.java   |  2 +-
 .../ssl/TestReloadingX509TrustManager.java        | 18 ++++++++++++++++++
 3 files changed, 23 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2b5d602/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
index 4e59010..a01d11a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
@@ -202,8 +202,10 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
           SSL_TRUSTSTORE_PASSWORD_TPL_KEY);
       String truststorePassword = getPassword(conf, passwordProperty, "");
       if (truststorePassword.isEmpty()) {
-        throw new GeneralSecurityException("The property '" + passwordProperty +
-            "' has not been set in the ssl configuration file.");
+        // An empty trust store password is legal; the trust store password
+        // is only required when writing to a trust store. Otherwise it's
+        // an optional integrity check.
+        truststorePassword = null;
       }
       long truststoreReloadInterval =
           conf.getLong(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2b5d602/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java
index 597f8d7..2d3afea 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java
@@ -167,7 +167,7 @@ public final class ReloadingX509TrustManager
     KeyStore ks = KeyStore.getInstance(type);
     FileInputStream in = new FileInputStream(file);
     try {
-      ks.load(in, password.toCharArray());
+      ks.load(in, (password == null) ? null : password.toCharArray());
       lastLoaded = file.lastModified();
       LOG.debug("Loaded truststore '" + file + "'");
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2b5d602/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java
index bf058cd..3fb203e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestReloadingX509TrustManager.java
@@ -199,4 +199,22 @@ public class TestReloadingX509TrustManager {
     }, reloadInterval, 10 * 1000);
   }
 
+  /** No password when accessing a trust store is legal. */
+  @Test
+  public void testNoPassword() throws Exception {
+    KeyPair kp = generateKeyPair("RSA");
+    cert1 = generateCertificate("CN=Cert1", kp, 30, "SHA1withRSA");
+    cert2 = generateCertificate("CN=Cert2", kp, 30, "SHA1withRSA");
+    String truststoreLocation = BASEDIR + "/testreload.jks";
+    createTrustStore(truststoreLocation, "password", "cert1", cert1);
+
+    final ReloadingX509TrustManager tm =
+        new ReloadingX509TrustManager("jks", truststoreLocation, null, 10);
+    try {
+      tm.init();
+      assertEquals(1, tm.getAcceptedIssuers().length);
+    } finally {
+      tm.destroy();
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/50] [abbrv] hadoop git commit: HDFS-11201. Spelling errors in the logging, help, assertions and exception messages. Contributed by Grant Sohn.

Posted by st...@apache.org.
HDFS-11201. Spelling errors in the logging, help, assertions and exception messages. Contributed by Grant Sohn.


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

Branch: refs/heads/HADOOP-13345
Commit: b9522e86a55564c2ccb5ca3f1ca871965cbe74de
Parents: 291df5c
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Mon Dec 5 09:37:12 2016 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Mon Dec 5 10:48:25 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java     | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java | 4 ++--
 .../main/java/org/apache/hadoop/lib/server/ServerException.java  | 2 +-
 .../java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java   | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/DFSUtil.java            | 2 +-
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java  | 2 +-
 .../hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java      | 2 +-
 .../hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java | 2 +-
 .../hadoop/hdfs/server/diskbalancer/command/QueryCommand.java    | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java  | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java  | 2 +-
 .../java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java   | 4 ++--
 .../server/namenode/web/resources/NamenodeWebHdfsMethods.java    | 2 +-
 13 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 5783f90..aabd6fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1052,7 +1052,7 @@ public class DFSInputStream extends FSInputStream
             reader.getNetworkDistance(), nread);
         if (nread != len) {
           throw new IOException("truncated return from reader.read(): " +
-              "excpected " + len + ", got " + nread);
+              "expected " + len + ", got " + nread);
         }
         DFSClientFaultInjector.get().readFromDatanodeDelay();
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
index 51ad08f..db064e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
@@ -57,11 +57,11 @@ public class LongBitFormat implements Serializable {
   public long combine(long value, long record) {
     if (value < MIN) {
       throw new IllegalArgumentException(
-          "Illagal value: " + NAME + " = " + value + " < MIN = " + MIN);
+          "Illegal value: " + NAME + " = " + value + " < MIN = " + MIN);
     }
     if (value > MAX) {
       throw new IllegalArgumentException(
-          "Illagal value: " + NAME + " = " + value + " > MAX = " + MAX);
+          "Illegal value: " + NAME + " = " + value + " > MAX = " + MAX);
     }
     return (record & ~MASK) | (value << OFFSET);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
index e3759ce..fdca64e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
@@ -38,7 +38,7 @@ public class ServerException extends XException {
     S04("Service [{0}] does not implement declared interface [{1}]"),
     S05("[{0}] is not a file"),
     S06("Could not load file [{0}], {1}"),
-    S07("Could not instanciate service class [{0}], {1}"),
+    S07("Could not instantiate service class [{0}], {1}"),
     S08("Could not load service classes, {0}"),
     S09("Could not set service [{0}] programmatically -server shutting down-, {1}"),
     S10("Service [{0}] requires service [{1}]"),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
index e26fac5..2d8f676 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
@@ -130,7 +130,7 @@ class OpenFileCtxCache {
           }
           toEvict = openFileMap.remove(pairs.getKey());
           Preconditions.checkState(toEvict == pairs.getValue(),
-              "The deleted entry is not the same as odlest found.");
+              "The deleted entry is not the same as oldest found.");
         }
       }
       openFileMap.put(h, context);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 23166e2..4664699 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -1364,7 +1364,7 @@ public class DFSUtil {
         DFSConfigKeys.DFS_HTTP_POLICY_DEFAULT);
     HttpConfig.Policy policy = HttpConfig.Policy.fromString(policyStr);
     if (policy == null) {
-      throw new HadoopIllegalArgumentException("Unregonized value '"
+      throw new HadoopIllegalArgumentException("Unrecognized value '"
           + policyStr + "' for " + DFSConfigKeys.DFS_HTTP_POLICY_KEY);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index e60703b..05d538a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2772,7 +2772,7 @@ public class BlockManager implements BlockStatsMXBean {
       throws IOException {
     for (ReportedBlockInfo rbi : rbis) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Processing previouly queued message " + rbi);
+        LOG.debug("Processing previously queued message " + rbi);
       }
       if (rbi.getReportedState() == null) {
         // This is a DELETE_BLOCK request

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 8323140..69dc9f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -836,7 +836,7 @@ class BlockPoolSlice {
     } catch (Exception e) {
       // Any exception we need to revert back to read from disk
       // Log the error and return false
-      LOG.info("Exception occured while reading the replicas cache file: "
+      LOG.info("Exception occurred while reading the replicas cache file: "
           + replicaFile.getPath(), e );
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 08564de..d41f9c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -1292,7 +1292,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
     try {
       fileNames = IOUtils.listDirectory(dir, BlockDirFilter.INSTANCE);
     } catch (IOException ioe) {
-      LOG.warn("Exception occured while compiling report: ", ioe);
+      LOG.warn("Exception occurred while compiling report: ", ioe);
       // Initiate a check on disk failure.
       dataset.datanode.checkDiskErrorAsync();
       // Ignore this directory and proceed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
index a8adcbd..9e60e48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -97,7 +97,7 @@ public class QueryCommand extends Command {
     String header = "Query Plan queries a given data node about the " +
         "current state of disk balancer execution.\n\n";
 
-    String footer = "\nQuery command retrievs the plan ID and the current " +
+    String footer = "\nQuery command retrieves the plan ID and the current " +
         "running state. ";
 
     HelpFormatter helpFormatter = new HelpFormatter();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
index f2a1ee5..41ec8e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
@@ -134,7 +134,7 @@ final class FSDirTruncateOp {
       if (!onBlockBoundary) {
         // Open file for write, but don't log into edits
         long lastBlockDelta = file.computeFileSize() - newLength;
-        assert lastBlockDelta > 0 : "delta is 0 only if on block bounday";
+        assert lastBlockDelta > 0 : "delta is 0 only if on block boundary";
         truncateBlock = prepareFileForTruncate(fsn, iip, clientName,
             clientMachine, lastBlockDelta, null);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 2990344..11cdbc6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -1082,7 +1082,7 @@ public class FSEditLogLoader {
       boolean removed = FSDirWriteFileOp.unprotectedRemoveBlock(
           fsDir, path, iip, file, oldBlock);
       if (!removed && !(op instanceof UpdateBlocksOp)) {
-        throw new IOException("Trying to delete non-existant block " + oldBlock);
+        throw new IOException("Trying to delete non-existent block " + oldBlock);
       }
     } else if (newBlocks.length > oldBlocks.length) {
       final boolean isStriped = ecPolicy != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index d6dd8ee..f79130d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -1025,7 +1025,7 @@ public class NNStorage extends Storage implements Closeable,
     
     if (!blockpoolID.equals("") && !blockpoolID.equals(bpid)) {
       throw new InconsistentFSStateException(storage,
-          "Unexepcted blockpoolID " + bpid + " . Expected " + blockpoolID);
+          "Unexpected blockpoolID " + bpid + " . Expected " + blockpoolID);
     }
     setBlockPoolID(bpid);
   }
@@ -1154,4 +1154,4 @@ public class NNStorage extends Storage implements Closeable,
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9522e86/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 107d4ed..2c31cd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -219,7 +219,7 @@ public class NamenodeWebHdfsMethods {
       final String remoteAddr) throws IOException {
     FSNamesystem fsn = namenode.getNamesystem();
     if (fsn == null) {
-      throw new IOException("Namesystem has not been intialized yet.");
+      throw new IOException("Namesystem has not been initialized yet.");
     }
     final BlockManager bm = fsn.getBlockManager();
     


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/50] [abbrv] hadoop git commit: HADOOP-13675. Bug in return value for delete() calls in WASB. Contributed by Dushyanth

Posted by st...@apache.org.
HADOOP-13675. Bug in return value for delete() calls in WASB. Contributed by Dushyanth


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

Branch: refs/heads/HADOOP-13345
Commit: 15dd1f3381069c5fdc6690e3ab1907a133ba14bf
Parents: 8c46808
Author: Mingliang Liu <li...@apache.org>
Authored: Mon Dec 5 12:04:07 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Mon Dec 5 12:04:07 2016 -0800

----------------------------------------------------------------------
 .../fs/azure/AzureNativeFileSystemStore.java    |  31 +++--
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |  25 ++--
 .../hadoop/fs/azure/NativeFileSystemStore.java  |  23 +++-
 ...estNativeAzureFileSystemConcurrencyLive.java | 119 +++++++++++++++++++
 4 files changed, 176 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/15dd1f33/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index 3e864a4..ac6c514 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -2045,10 +2045,10 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    *          The key to search for.
    * @return The wanted directory, or null if not found.
    */
-  private static FileMetadata getDirectoryInList(
+  private static FileMetadata getFileMetadataInList(
       final Iterable<FileMetadata> list, String key) {
     for (FileMetadata current : list) {
-      if (current.isDir() && current.getKey().equals(key)) {
+      if (current.getKey().equals(key)) {
         return current;
       }
     }
@@ -2114,7 +2114,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
           // Add the metadata to the list, but remove any existing duplicate
           // entries first that we may have added by finding nested files.
-          FileMetadata existing = getDirectoryInList(fileMetadata, blobKey);
+          FileMetadata existing = getFileMetadataInList(fileMetadata, blobKey);
           if (existing != null) {
             fileMetadata.remove(existing);
           }
@@ -2141,7 +2141,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
           // Add the directory metadata to the list only if it's not already
           // there.
-          if (getDirectoryInList(fileMetadata, dirKey) == null) {
+          if (getFileMetadataInList(fileMetadata, dirKey) == null) {
             fileMetadata.add(directoryMetadata);
           }
 
@@ -2249,7 +2249,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
           // Add the directory metadata to the list only if it's not already
           // there.
-          FileMetadata existing = getDirectoryInList(aFileMetadataList, blobKey);
+          FileMetadata existing = getFileMetadataInList(aFileMetadataList, blobKey);
           if (existing != null) {
             aFileMetadataList.remove(existing);
           }
@@ -2278,7 +2278,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
             // absolute path is being used or not.
             String dirKey = normalizeKey(directory);
 
-            if (getDirectoryInList(aFileMetadataList, dirKey) == null) {
+            if (getFileMetadataInList(aFileMetadataList, dirKey) == null) {
               // Reached the targeted listing depth. Return metadata for the
               // directory using default permissions.
               //
@@ -2376,18 +2376,24 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     }
   }
 
+  /**
+   * API implementation to delete a blob in the back end azure storage.
+   */
   @Override
-  public void delete(String key, SelfRenewingLease lease) throws IOException {
+  public boolean delete(String key, SelfRenewingLease lease) throws IOException {
     try {
       if (checkContainer(ContainerAccessType.ReadThenWrite) == ContainerState.DoesntExist) {
         // Container doesn't exist, no need to do anything
-        return;
+        return true;
       }
 
       // Get the blob reference and delete it.
       CloudBlobWrapper blob = getBlobReference(key);
       if (blob.exists(getInstrumentedContext())) {
         safeDelete(blob, lease);
+        return true;
+      } else {
+        return false;
       }
     } catch (Exception e) {
       // Re-throw as an Azure storage exception.
@@ -2395,10 +2401,13 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     }
   }
 
+  /**
+   * API implementation to delete a blob in the back end azure storage.
+   */
   @Override
-  public void delete(String key) throws IOException {
+  public boolean delete(String key) throws IOException {
     try {
-      delete(key, null);
+      return delete(key, null);
     } catch (IOException e) {
       Throwable t = e.getCause();
       if(t != null && t instanceof StorageException) {
@@ -2407,7 +2416,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
           SelfRenewingLease lease = null;
           try {
             lease = acquireLease(key);
-            delete(key, lease);
+            return delete(key, lease);
           } catch (AzureException e3) {
             LOG.warn("Got unexpected exception trying to acquire lease on "
                 + key + "." + e3.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15dd1f33/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index 54eb90f..307c2fa 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -1765,8 +1765,11 @@ public class NativeAzureFileSystem extends FileSystem {
       }
 
       try {
-        store.delete(key);
-        instrumentation.fileDeleted();
+        if (store.delete(key)) {
+          instrumentation.fileDeleted();
+        } else {
+          return false;
+        }
       } catch(IOException e) {
 
         Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(e);
@@ -1885,7 +1888,7 @@ public class NativeAzureFileSystem extends FileSystem {
       }
 
       // Delete the current directory
-      if (!deleteFile(metaFile.getKey(), metaFile.isDir())) {
+      if (store.retrieveMetadata(metaFile.getKey()) != null && !deleteFile(metaFile.getKey(), metaFile.isDir())) {
         LOG.error("Failed delete directory {}", f.toString());
         return false;
       }
@@ -1913,11 +1916,15 @@ public class NativeAzureFileSystem extends FileSystem {
   @VisibleForTesting
   boolean deleteFile(String key, boolean isDir) throws IOException {
     try {
-      store.delete(key);
-      if (isDir) {
-        instrumentation.directoryDeleted();
+      if (store.delete(key)) {
+        if (isDir) {
+          instrumentation.directoryDeleted();
+        } else {
+          instrumentation.fileDeleted();
+        }
+        return true;
       } else {
-        instrumentation.fileDeleted();
+        return false;
       }
     } catch(IOException e) {
       Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(e);
@@ -1929,8 +1936,6 @@ public class NativeAzureFileSystem extends FileSystem {
 
       throw e;
     }
-
-    return true;
   }
 
   @Override
@@ -2790,6 +2795,8 @@ public class NativeAzureFileSystem extends FileSystem {
         throws IOException {
 
       LOG.debug("Deleting dangling file {}", file.getKey());
+      // Not handling delete return type as false return essentially
+      // means its a no-op for the caller
       store.delete(file.getKey());
       store.delete(tempFile.getKey());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15dd1f33/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
index acdd3d6..454a5df 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
@@ -74,7 +74,16 @@ interface NativeFileSystemStore {
   void changePermissionStatus(String key, PermissionStatus newPermission)
       throws AzureException;
 
-  void delete(String key) throws IOException;
+  /**
+   * API to delete a blob in the back end azure storage.
+   * @param key - key to the blob being deleted.
+   * @return return true when delete is successful, false if
+   * blob cannot be found or delete is not possible without
+   * exception.
+   * @throws IOException Exception encountered while deleting in
+   * azure storage.
+   */
+  boolean delete(String key) throws IOException;
 
   void rename(String srcKey, String dstKey) throws IOException;
 
@@ -104,7 +113,17 @@ interface NativeFileSystemStore {
   void updateFolderLastModifiedTime(String key, Date lastModified,
       SelfRenewingLease folderLease) throws AzureException;
 
-  void delete(String key, SelfRenewingLease lease) throws IOException;
+  /**
+   * API to delete a blob in the back end azure storage.
+   * @param key - key to the blob being deleted.
+   * @param lease - Active lease on the blob.
+   * @return return true when delete is successful, false if
+   * blob cannot be found or delete is not possible without
+   * exception.
+   * @throws IOException Exception encountered while deleting in
+   * azure storage.
+   */
+  boolean delete(String key, SelfRenewingLease lease) throws IOException;
       
   SelfRenewingLease acquireLease(String key) throws AzureException;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15dd1f33/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java
new file mode 100644
index 0000000..ec72cce
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java
@@ -0,0 +1,119 @@
+/**
+ * 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.azure;
+
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Test;
+
+/***
+ * Test class to hold all Live Azure storage concurrency tests.
+ */
+public class TestNativeAzureFileSystemConcurrencyLive
+    extends AbstractWasbTestBase {
+
+  private static final int TEST_COUNT = 102;
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  /**
+   * Test multi-threaded deletes in WASB. Expected behavior is one of the thread
+   * should be to successfully delete the file and return true and all other
+   * threads need to return false.
+   */
+  @Test
+  public void testMultiThreadedDeletes() throws Exception {
+    Path testFile = new Path("test.dat");
+    fs.create(testFile).close();
+
+    int threadCount = TEST_COUNT;
+    DeleteHelperThread[] helperThreads = new DeleteHelperThread[threadCount];
+
+    for (int i = 0; i < threadCount; i++) {
+      helperThreads[i] = new DeleteHelperThread(fs, testFile);
+    }
+
+    Thread[] threads = new Thread[threadCount];
+
+    for (int i = 0; i < threadCount; i++) {
+      threads[i] = new Thread(helperThreads[i]);
+      threads[i].start();
+    }
+
+    for (int i = 0; i < threadCount; i++) {
+      threads[i].join();
+    }
+
+    boolean deleteSuccess = false;
+
+    for (int i = 0; i < threadCount; i++) {
+
+      Assert.assertFalse("child thread has exception : " + helperThreads[i].getException(),
+          helperThreads[i].getExceptionEncounteredFlag());
+
+      if (deleteSuccess) {
+        Assert.assertFalse("More than one thread delete() retuhelperThreads[i].getDeleteSuccess()",
+            helperThreads[i].getExceptionEncounteredFlag());
+      } else {
+        deleteSuccess = helperThreads[i].getDeleteSuccess();
+      }
+    }
+
+    Assert.assertTrue("No successfull delete found", deleteSuccess);
+  }
+}
+
+class DeleteHelperThread implements Runnable {
+
+  private FileSystem fs;
+  private Path p;
+  private boolean deleteSuccess;
+  private boolean exceptionEncountered;
+  private Exception ex;
+
+  public DeleteHelperThread(FileSystem fs, Path p) {
+    this.fs = fs;
+    this.p = p;
+  }
+
+  public void run() {
+    try {
+      deleteSuccess = fs.delete(p, false);
+    } catch (Exception ioEx) {
+      exceptionEncountered = true;
+      this.ex = ioEx;
+    }
+  }
+
+  public boolean getDeleteSuccess() {
+    return deleteSuccess;
+  }
+
+  public boolean getExceptionEncounteredFlag() {
+    return exceptionEncountered;
+  }
+
+  public Exception getException() {
+    return ex;
+  }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[47/50] [abbrv] hadoop git commit: HDFS-11223. Fix typos in HttpFs documentations. Contributed by Yiqun Lin.

Posted by st...@apache.org.
HDFS-11223. Fix typos in HttpFs documentations. Contributed by Yiqun Lin.


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

Branch: refs/heads/HADOOP-13345
Commit: 4c2cf5560f6d952cfa36ef656f0b04dc3150f8b3
Parents: 74d0066
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 8 20:52:24 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 8 20:52:24 2016 +0900

----------------------------------------------------------------------
 .../hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm     | 2 +-
 .../hadoop-hdfs-httpfs/src/site/markdown/index.md              | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c2cf556/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
index 0cb89de..4b66732 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
@@ -50,7 +50,7 @@ IMPORTANT: Replace `#HTTPFSUSER#` with the Unix user that will start the HttpFS
 Restart Hadoop
 --------------
 
-You need to restart Hadoop for the proxyuser configuration ot become active.
+You need to restart Hadoop for the proxyuser configuration to become active.
 
 Start/Stop HttpFS
 -----------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c2cf556/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md
index 750b7f4..145feb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/index.md
@@ -15,7 +15,7 @@
 Hadoop HDFS over HTTP - Documentation Sets
 ==========================================
 
-HttpFS is a server that provides a REST HTTP gateway supporting all HDFS File System operations (read and write). And it is inteoperable with the **webhdfs** REST HTTP API.
+HttpFS is a server that provides a REST HTTP gateway supporting all HDFS File System operations (read and write). And it is interoperable with the **webhdfs** REST HTTP API.
 
 HttpFS can be used to transfer data between clusters running different versions of Hadoop (overcoming RPC versioning issues), for example using Hadoop DistCP.
 
@@ -23,9 +23,9 @@ HttpFS can be used to access data in HDFS on a cluster behind of a firewall (the
 
 HttpFS can be used to access data in HDFS using HTTP utilities (such as curl and wget) and HTTP libraries Perl from other languages than Java.
 
-The **webhdfs** client FileSytem implementation can be used to access HttpFS using the Hadoop filesystem command (`hadoop fs`) line tool as well as from Java applications using the Hadoop FileSystem Java API.
+The **webhdfs** client FileSystem implementation can be used to access HttpFS using the Hadoop filesystem command (`hadoop fs`) line tool as well as from Java applications using the Hadoop FileSystem Java API.
 
-HttpFS has built-in security supporting Hadoop pseudo authentication and HTTP SPNEGO Kerberos and other pluggable authentication mechanims. It also provides Hadoop proxy user support.
+HttpFS has built-in security supporting Hadoop pseudo authentication and HTTP SPNEGO Kerberos and other pluggable authentication mechanisms. It also provides Hadoop proxy user support.
 
 How Does HttpFS Works?
 ----------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/50] [abbrv] hadoop git commit: YARN-5963. Spelling errors in logging and exceptions for node manager, client, web-proxy, common, and app history code (gsohn via rkanter)

Posted by st...@apache.org.
YARN-5963. Spelling errors in logging and exceptions for node manager, client, web-proxy, common, and app history code (gsohn via rkanter)


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

Branch: refs/heads/HADOOP-13345
Commit: 72fe54684198b7df5c5fb2114616dff6d17a4402
Parents: a793cec
Author: Robert Kanter <rk...@apache.org>
Authored: Wed Dec 7 15:07:25 2016 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Wed Dec 7 15:07:25 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java | 2 +-
 .../main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java  | 2 +-
 .../apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java   | 2 +-
 .../src/main/java/org/apache/hadoop/yarn/util/FSDownload.java    | 2 +-
 .../apache/hadoop/registry/client/impl/zk/RegistrySecurity.java  | 2 +-
 .../ApplicationHistoryManagerOnTimelineStore.java                | 2 +-
 .../yarn/server/nodemanager/WindowsSecureContainerExecutor.java  | 4 ++--
 .../yarn/server/nodemanager/containermanager/AuxServices.java    | 4 ++--
 .../nodemanager/containermanager/ContainerManagerImpl.java       | 2 +-
 .../containermanager/localizer/ResourceLocalizationService.java  | 2 +-
 .../containermanager/logaggregation/AppLogAggregatorImpl.java    | 2 +-
 .../org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java    | 2 +-
 .../java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java | 2 +-
 13 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 25e3a46..9b21ff8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -240,7 +240,7 @@ public class LogsCLI extends Configured implements Tool {
       if (appState == YarnApplicationState.NEW
           || appState == YarnApplicationState.NEW_SAVING
           || appState == YarnApplicationState.SUBMITTED) {
-        System.err.println("Logs are not avaiable right now.");
+        System.err.println("Logs are not available right now.");
         return -1;
       }
     } catch (IOException | YarnException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
index 7779ddf..2508c3e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
@@ -379,7 +379,7 @@ public class RMAdminCLI extends HAAdmin {
     }
     if (nodesDecommissioning) {
       System.out.println("Graceful decommissioning not completed in " + timeout
-          + " seconds, issueing forceful decommissioning command.");
+          + " seconds, issuing forceful decommissioning command.");
       RefreshNodesRequest forcefulRequest = RefreshNodesRequest
           .newInstance(DecommissionType.FORCEFUL);
       adminProtocol.refreshNodes(forcefulRequest);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
index 4103fd6..b275a94 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -451,7 +451,7 @@ public class TimelineClientImpl extends TimelineClient {
       String errMessage = "TimelineClient has reached to max retry times : "
           + this.maxServiceRetries
           + ", but failed to fetch timeline service address. Please verify"
-          + " Timeline Auxillary Service is configured in all the NMs";
+          + " Timeline Auxiliary Service is configured in all the NMs";
       LOG.error(errMessage);
       throw new YarnException(errMessage);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
index de18dc6..f34c16c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
@@ -259,7 +259,7 @@ public class FSDownload implements Callable<Path> {
     if (resource.getVisibility() == LocalResourceVisibility.PUBLIC) {
       if (!isPublic(sourceFs, sCopy, sStat, statCache)) {
         throw new IOException("Resource " + sCopy +
-            " is not publicly accessable and as such cannot be part of the" +
+            " is not publicly accessible and as such cannot be part of the" +
             " public cache.");
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
index 39325a3..a3ec77a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
@@ -358,7 +358,7 @@ public class RegistrySecurity extends AbstractService {
    * @return the system principals
    */
   public List<ACL> getSystemACLs() {
-    Preconditions.checkNotNull(systemACLs, "registry security is unitialized");
+    Preconditions.checkNotNull(systemACLs, "registry security is uninitialized");
     return Collections.unmodifiableList(systemACLs);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
index 3249c8f..54893ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -722,7 +722,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
              app.appReport.getApplicationId())) {
            throw new AuthorizationException("User "
                + UserGroupInformation.getCurrentUser().getShortUserName()
-               + " does not have privilage to see this application "
+               + " does not have privilege to see this application "
                + app.appReport.getApplicationId());
          }
        } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
index 9e38151..7c124c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
@@ -509,7 +509,7 @@ public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
               output.append(buf, 0, nRead);
             }
           } catch (Throwable t) {
-            LOG.error("Error occured reading the process stdout", t);
+            LOG.error("Error occurred reading the process stdout", t);
           }
         }
       };
@@ -722,7 +722,7 @@ public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
       }
       catch(Throwable e) {
         LOG.warn(String.format(
-            "An exception occured during the cleanup of localizer job %s:%n%s",
+            "An exception occurred during the cleanup of localizer job %s:%n%s",
             localizerPid,
             org.apache.hadoop.util.StringUtils.stringifyException(e)));
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
index 171b20d..c0e1f5a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
@@ -146,7 +146,7 @@ public class AuxServices extends AbstractService
         }
         // TODO better use s.getName()?
         if(!sName.equals(s.getName())) {
-          LOG.warn("The Auxilurary Service named '"+sName+"' in the "
+          LOG.warn("The Auxiliary Service named '"+sName+"' in the "
               +"configuration is for "+s.getClass()+" which has "
               +"a name of '"+s.getName()+"'. Because these are "
               +"not the same tools trying to send ServiceData and read "
@@ -283,4 +283,4 @@ public class AuxServices extends AbstractService
         : "The auxService name is " + service.getName())
         + " and it got an error at event: " + eventType, th);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 9546a30..e4d9346 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -1673,7 +1673,7 @@ public class ContainerManagerImpl extends CompositeService implements
     authorizeUser(remoteUgi, nmTokenIdentifier);
     if (!nmTokenIdentifier.getApplicationAttemptId().getApplicationId()
         .equals(containerId.getApplicationAttemptId().getApplicationId())) {
-      throw new YarnException("ApplicationMaster not autorized to perform " +
+      throw new YarnException("ApplicationMaster not authorized to perform " +
           "["+ op + "] on Container [" + containerId + "]!!");
     }
     Container container = context.getContainers().get(containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 71971c7..e4d74f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -1128,7 +1128,7 @@ public class ResourceLocalizationService extends CompositeService
           LOG.error("local path for PRIVATE localization could not be " +
             "found. Disks might have failed.", e);
         } catch (IllegalArgumentException e) {
-          LOG.error("Inorrect path for PRIVATE localization."
+          LOG.error("Incorrect path for PRIVATE localization."
               + next.getResource().getFile(), e);
         } catch (URISyntaxException e) {
           LOG.error(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index 872b805..07275b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -494,7 +494,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       doAppLogAggregation();
     } catch (Exception e) {
       // do post clean up of log directories on any exception
-      LOG.error("Error occured while aggregating the log for the application "
+      LOG.error("Error occurred while aggregating the log for the application "
           + appId, e);
       doAppLogAggregationPostCleanUp();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
index c656742..f25f4c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
@@ -49,7 +49,7 @@ public class ProxyUriUtils {
   
   private static String uriEncode(Object o) {
     try {
-      assert (o != null) : "o canot be null";
+      assert (o != null) : "o cannot be null";
       return URLEncoder.encode(o.toString(), "UTF-8");
     } catch (UnsupportedEncodingException e) {
       //This should never happen

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72fe5468/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
index 5935327..71679cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
@@ -62,7 +62,7 @@ public class WebAppProxy extends AbstractService {
     } else if ("kerberos".equals(auth)) {
       isSecurityEnabled = true;
     } else {
-      LOG.warn("Unrecongized attribute value for " +
+      LOG.warn("Unrecognized attribute value for " +
           CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION +
           " of " + auth);
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/50] [abbrv] hadoop git commit: YARN-5184. Fix up incompatible changes introduced on ContainerStatus and NodeReport. Contributed by Sangjin Lee.

Posted by st...@apache.org.
YARN-5184. Fix up incompatible changes introduced on ContainerStatus and NodeReport. Contributed by Sangjin Lee.


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

Branch: refs/heads/HADOOP-13345
Commit: a7288da595fdf56c3ccd45c0b6ed2e3efaa043a4
Parents: c8d0a04
Author: Junping Du <ju...@apache.org>
Authored: Tue Dec 6 14:36:41 2016 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Tue Dec 6 14:36:41 2016 -0800

----------------------------------------------------------------------
 .../yarn/api/records/ContainerStatus.java       | 40 ++++++++++++++++----
 .../hadoop/yarn/api/records/NodeReport.java     | 12 ++++--
 2 files changed, 41 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7288da5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
index 839fd04..d7c75f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
@@ -85,11 +85,17 @@ public abstract class ContainerStatus {
    */
   @Public
   @Evolving
-  public abstract ExecutionType getExecutionType();
+  public ExecutionType getExecutionType() {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   @Private
   @Unstable
-  public abstract void setExecutionType(ExecutionType executionType);
+  public void setExecutionType(ExecutionType executionType) {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   /**
    * Get the <code>ContainerState</code> of the container.
@@ -148,11 +154,17 @@ public abstract class ContainerStatus {
    */
   @Public
   @Unstable
-  public abstract Resource getCapability();
+  public Resource getCapability() {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   @Private
   @Unstable
-  public abstract void setCapability(Resource capability);
+  public void setCapability(Resource capability) {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   /**
    * Get all the IP addresses with which the container run.
@@ -160,11 +172,17 @@ public abstract class ContainerStatus {
    */
   @Public
   @Unstable
-  public abstract List<String> getIPs();
+  public List<String> getIPs() {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   @Private
   @Unstable
-  public abstract void setIPs(List<String> ips);
+  public void setIPs(List<String> ips) {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   /**
    * Get the hostname where the container runs.
@@ -172,9 +190,15 @@ public abstract class ContainerStatus {
    */
   @Public
   @Unstable
-  public abstract String getHost();
+  public String getHost() {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   @Private
   @Unstable
-  public abstract void setHost(String host);
+  public void setHost(String host) {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7288da5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
index 412010a..885a3b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
@@ -203,12 +203,18 @@ public abstract class NodeReport {
    */
   @Public
   @Stable
-  public abstract ResourceUtilization getAggregatedContainersUtilization();
+  public ResourceUtilization getAggregatedContainersUtilization() {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   @Private
   @Unstable
-  public abstract void setAggregatedContainersUtilization(ResourceUtilization
-      containersUtilization);
+  public void setAggregatedContainersUtilization(ResourceUtilization
+      containersUtilization) {
+    throw new UnsupportedOperationException(
+        "subclass must implement this method");
+  }
 
   /**
    * Get node resource utilization


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/50] [abbrv] hadoop git commit: YARN-5932. Retrospect moveApplicationToQueue in align with YARN-5611. Contributed by Sunil G.

Posted by st...@apache.org.
YARN-5932. Retrospect moveApplicationToQueue in align with YARN-5611. Contributed by Sunil G.


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

Branch: refs/heads/HADOOP-13345
Commit: 563480dccd0136d82730f4228f1df44449ed5822
Parents: a7288da
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Wed Dec 7 10:39:14 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Wed Dec 7 10:39:14 2016 +0530

----------------------------------------------------------------------
 .../server/resourcemanager/ClientRMService.java |  18 +--
 .../server/resourcemanager/RMAppManager.java    | 110 +++++++++++++++++--
 .../resourcemanager/RMAppManagerEvent.java      |  11 ++
 .../resourcemanager/RMAppManagerEventType.java  |   3 +-
 .../resourcemanager/rmapp/RMAppEventType.java   |   1 -
 .../server/resourcemanager/rmapp/RMAppImpl.java |  48 +-------
 .../resourcemanager/rmapp/RMAppMoveEvent.java   |  44 --------
 .../scheduler/AbstractYarnScheduler.java        |  18 ++-
 .../scheduler/YarnScheduler.java                |  11 ++
 .../scheduler/capacity/AbstractCSQueue.java     |   9 ++
 .../scheduler/capacity/CSQueue.java             |  10 ++
 .../scheduler/capacity/CapacityScheduler.java   |  27 ++++-
 .../scheduler/capacity/LeafQueue.java           |  19 +++-
 .../scheduler/capacity/ParentQueue.java         |  29 +++--
 .../scheduler/fair/FairScheduler.java           |  36 +++++-
 .../resourcemanager/TestMoveApplication.java    |  12 +-
 .../TestCapacitySchedulerNodeLabelUpdate.java   |   1 +
 17 files changed, 274 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 0db775f..1bc40c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -152,7 +152,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSyst
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppKillByClientEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMoveEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
@@ -174,8 +173,6 @@ import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.UTCClock;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.SettableFuture;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
 
@@ -1191,23 +1188,18 @@ public class ClientRMService extends AbstractService implements
     }
     
     // Moves only allowed when app is in a state that means it is tracked by
-    // the scheduler
-    if (EnumSet.of(RMAppState.NEW, RMAppState.NEW_SAVING, RMAppState.FAILED,
-        RMAppState.FINAL_SAVING, RMAppState.FINISHING, RMAppState.FINISHED,
-        RMAppState.KILLED, RMAppState.KILLING, RMAppState.FAILED)
-        .contains(application.getState())) {
+    // the scheduler. Introducing SUBMITTED state also to this list as there
+    // could be a corner scenario that app may not be in Scheduler in SUBMITTED
+    // state.
+    if (!ACTIVE_APP_STATES.contains(application.getState())) {
       String msg = "App in " + application.getState() + " state cannot be moved.";
       RMAuditLogger.logFailure(callerUGI.getShortUserName(),
           AuditConstants.MOVE_APP_REQUEST, "UNKNOWN", "ClientRMService", msg);
       throw new YarnException(msg);
     }
 
-    SettableFuture<Object> future = SettableFuture.create();
-    this.rmContext.getDispatcher().getEventHandler().handle(
-        new RMAppMoveEvent(applicationId, request.getTargetQueue(), future));
-    
     try {
-      Futures.get(future, YarnException.class);
+      this.rmAppManager.moveApplicationAcrossQueue(applicationId, request.getTargetQueue());
     } catch (YarnException ex) {
       RMAuditLogger.logFailure(callerUGI.getShortUserName(),
           AuditConstants.MOVE_APP_REQUEST, "UNKNOWN", "ClientRMService",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index ce3da06..4d628ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -498,17 +498,26 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     ApplicationId applicationId = event.getApplicationId();
     LOG.debug("RMAppManager processing event for " 
         + applicationId + " of type " + event.getType());
-    switch(event.getType()) {
-      case APP_COMPLETED: 
-      {
-        finishApplication(applicationId);
-        logApplicationSummary(applicationId);
-        checkAppNumCompletedLimit(); 
-      } 
+    switch (event.getType()) {
+    case APP_COMPLETED :
+      finishApplication(applicationId);
+      logApplicationSummary(applicationId);
+      checkAppNumCompletedLimit();
       break;
-      default:
-        LOG.error("Invalid eventtype " + event.getType() + ". Ignoring!");
+    case APP_MOVE :
+      // moveAllApps from scheduler will fire this event for each of
+      // those applications which needed to be moved to a new queue.
+      // Use the standard move application api to do the same.
+      try {
+        moveApplicationAcrossQueue(applicationId,
+            event.getTargetQueueForMove());
+      } catch (YarnException e) {
+        LOG.warn("Move Application has failed: " + e.getMessage());
       }
+      break;
+    default :
+      LOG.error("Invalid eventtype " + event.getType() + ". Ignoring!");
+    }
   }
 
   // transaction method.
@@ -587,4 +596,87 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     rmContext.getSystemMetricsPublisher().appUpdated(app,
         System.currentTimeMillis());
   }
+
+  /**
+   * moveToQueue will invoke scheduler api to perform move queue operation.
+   *
+   * @param applicationId
+   *          Application Id.
+   * @param targetQueue
+   *          Target queue to which this app has to be moved.
+   * @throws YarnException
+   *           Handle exceptions.
+   */
+  public void moveApplicationAcrossQueue(ApplicationId applicationId, String targetQueue)
+      throws YarnException {
+    RMApp app = this.rmContext.getRMApps().get(applicationId);
+
+    // Capacity scheduler will directly follow below approach.
+    // 1. Do a pre-validate check to ensure that changes are fine.
+    // 2. Update this information to state-store
+    // 3. Perform real move operation and update in-memory data structures.
+    synchronized (applicationId) {
+      if (app.isAppInCompletedStates()) {
+        return;
+      }
+
+      String sourceQueue = app.getQueue();
+      // 1. pre-validate move application request to check for any access
+      // violations or other errors. If there are any violations, YarnException
+      // will be thrown.
+      rmContext.getScheduler().preValidateMoveApplication(applicationId,
+          targetQueue);
+
+      // 2. Update to state store with new queue and throw exception is failed.
+      updateAppDataToStateStore(targetQueue, app, false);
+
+      // 3. Perform the real move application
+      String queue = "";
+      try {
+        queue = rmContext.getScheduler().moveApplication(applicationId,
+            targetQueue);
+      } catch (YarnException e) {
+        // Revert to source queue since in-memory move has failed. Chances
+        // of this is very rare as we have already done the pre-validation.
+        updateAppDataToStateStore(sourceQueue, app, true);
+        throw e;
+      }
+
+      // update in-memory
+      if (queue != null && !queue.isEmpty()) {
+        app.setQueue(queue);
+      }
+    }
+
+    rmContext.getSystemMetricsPublisher().appUpdated(app,
+        System.currentTimeMillis());
+  }
+
+  private void updateAppDataToStateStore(String queue, RMApp app,
+      boolean toSuppressException) throws YarnException {
+    // Create a future object to capture exceptions from StateStore.
+    SettableFuture<Object> future = SettableFuture.create();
+
+    // Update new queue in Submission Context to update to StateStore.
+    app.getApplicationSubmissionContext().setQueue(queue);
+
+    ApplicationStateData appState = ApplicationStateData.newInstance(
+        app.getSubmitTime(), app.getStartTime(),
+        app.getApplicationSubmissionContext(), app.getUser(),
+        app.getCallerContext());
+    appState.setApplicationTimeouts(app.getApplicationTimeouts());
+    rmContext.getStateStore().updateApplicationStateSynchronously(appState,
+        false, future);
+
+    try {
+      Futures.get(future, YarnException.class);
+    } catch (YarnException ex) {
+      if (!toSuppressException) {
+        throw ex;
+      }
+      LOG.error("Statestore update failed for move application '"
+          + app.getApplicationId() + "' to queue '" + queue
+          + "' with below exception:" + ex.getMessage());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEvent.java
index f1a6781..0df3cab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEvent.java
@@ -24,13 +24,24 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
 public class RMAppManagerEvent extends AbstractEvent<RMAppManagerEventType> {
 
   private final ApplicationId appId;
+  private final String targetQueueForMove;
 
   public RMAppManagerEvent(ApplicationId appId, RMAppManagerEventType type) {
+    this(appId, "", type);
+  }
+
+  public RMAppManagerEvent(ApplicationId appId, String targetQueueForMove,
+      RMAppManagerEventType type) {
     super(type);
     this.appId = appId;
+    this.targetQueueForMove = targetQueueForMove;
   }
 
   public ApplicationId getApplicationId() {
     return this.appId;
   }
+
+  public String getTargetQueueForMove() {
+    return this.targetQueueForMove;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEventType.java
index 1b6a44c..7acf753 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEventType.java
@@ -19,5 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 public enum RMAppManagerEventType {
-  APP_COMPLETED
+  APP_COMPLETED,
+  APP_MOVE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
index 2b42638..aa5d6f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
@@ -23,7 +23,6 @@ public enum RMAppEventType {
   START,
   RECOVER,
   KILL,
-  MOVE, // Move app to a new queue
 
   // Source: Scheduler and RMAppManager
   APP_REJECTED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 0bf5f51..1f1586a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -71,7 +71,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
@@ -247,15 +246,11 @@ public class RMAppImpl implements RMApp, Recoverable {
         RMAppEventType.APP_REJECTED,
           new FinalSavingTransition(new AppRejectedTransition(),
             RMAppState.FAILED))
-    .addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
-        RMAppEventType.MOVE, new RMAppMoveTransition())
 
      // Transitions from SUBMITTED state
     .addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
     .addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
-        RMAppEventType.MOVE, new RMAppMoveTransition())
-    .addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
         RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
     .addTransition(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING,
         RMAppEventType.APP_REJECTED,
@@ -272,8 +267,6 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
     .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
-        RMAppEventType.MOVE, new RMAppMoveTransition())
-    .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
         RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
     .addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING,
         RMAppEventType.ATTEMPT_REGISTERED, new RMAppStateUpdateTransition(
@@ -301,8 +294,6 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
     .addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
-        RMAppEventType.MOVE, new RMAppMoveTransition())
-    .addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
         RMAppEventType.COLLECTOR_UPDATE, new RMAppCollectorUpdateTransition())
     .addTransition(RMAppState.RUNNING, RMAppState.FINAL_SAVING,
         RMAppEventType.ATTEMPT_UNREGISTERED,
@@ -338,7 +329,7 @@ public class RMAppImpl implements RMApp, Recoverable {
     // ignorable transitions
     .addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
         EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.KILL,
-          RMAppEventType.APP_NEW_SAVED, RMAppEventType.MOVE))
+          RMAppEventType.APP_NEW_SAVED))
 
      // Transitions from FINISHING state
     .addTransition(RMAppState.FINISHING, RMAppState.FINISHED,
@@ -353,7 +344,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       EnumSet.of(RMAppEventType.NODE_UPDATE,
         // ignore Kill/Move as we have already saved the final Finished state
         // in state store.
-        RMAppEventType.KILL, RMAppEventType.MOVE))
+        RMAppEventType.KILL))
 
      // Transitions from KILLING state
     .addTransition(RMAppState.KILLING, RMAppState.KILLING, 
@@ -383,7 +374,7 @@ public class RMAppImpl implements RMApp, Recoverable {
             RMAppEventType.NODE_UPDATE,
             RMAppEventType.ATTEMPT_REGISTERED,
             RMAppEventType.APP_UPDATE_SAVED,
-            RMAppEventType.KILL, RMAppEventType.MOVE))
+            RMAppEventType.KILL))
 
      // Transitions from FINISHED state
      // ignorable transitions
@@ -395,7 +386,7 @@ public class RMAppImpl implements RMApp, Recoverable {
             RMAppEventType.NODE_UPDATE,
             RMAppEventType.ATTEMPT_UNREGISTERED,
             RMAppEventType.ATTEMPT_FINISHED,
-            RMAppEventType.KILL, RMAppEventType.MOVE))
+            RMAppEventType.KILL))
 
      // Transitions from FAILED state
      // ignorable transitions
@@ -403,8 +394,7 @@ public class RMAppImpl implements RMApp, Recoverable {
         RMAppEventType.APP_RUNNING_ON_NODE,
         new AppRunningOnNodeTransition())
     .addTransition(RMAppState.FAILED, RMAppState.FAILED,
-        EnumSet.of(RMAppEventType.KILL, RMAppEventType.NODE_UPDATE,
-            RMAppEventType.MOVE))
+        EnumSet.of(RMAppEventType.KILL, RMAppEventType.NODE_UPDATE))
 
      // Transitions from KILLED state
      // ignorable transitions
@@ -417,7 +407,7 @@ public class RMAppImpl implements RMApp, Recoverable {
         EnumSet.of(RMAppEventType.APP_ACCEPTED,
             RMAppEventType.APP_REJECTED, RMAppEventType.KILL,
             RMAppEventType.ATTEMPT_FINISHED, RMAppEventType.ATTEMPT_FAILED,
-            RMAppEventType.NODE_UPDATE, RMAppEventType.MOVE))
+            RMAppEventType.NODE_UPDATE))
 
      .installTopology();
 
@@ -1077,32 +1067,6 @@ public class RMAppImpl implements RMApp, Recoverable {
     };
   }
 
-  /**
-   * Move an app to a new queue.
-   * This transition must set the result on the Future in the RMAppMoveEvent,
-   * either as an exception for failure or null for success, or the client will
-   * be left waiting forever.
-   */
-  private static final class RMAppMoveTransition extends RMAppTransition {
-    public void transition(RMAppImpl app, RMAppEvent event) {
-      RMAppMoveEvent moveEvent = (RMAppMoveEvent) event;
-      try {
-        app.queue = app.scheduler.moveApplication(app.applicationId,
-            moveEvent.getTargetQueue());
-      } catch (YarnException ex) {
-        moveEvent.getResult().setException(ex);
-        return;
-      }
-
-      app.rmContext.getSystemMetricsPublisher().appUpdated(app,
-          app.systemClock.getTime());
-
-      // TODO: Write out change to state store (YARN-1558)
-      // Also take care of RM failover
-      moveEvent.getResult().set(null);
-    }
-  }
-
   // synchronously recover attempt to ensure any incoming external events
   // to be processed after the attempt processes the recover event.
   private void recoverAppAttempts() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppMoveEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppMoveEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppMoveEvent.java
deleted file mode 100644
index 5fc63c9..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppMoveEvent.java
+++ /dev/null
@@ -1,44 +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.yarn.server.resourcemanager.rmapp;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-
-import com.google.common.util.concurrent.SettableFuture;
-
-public class RMAppMoveEvent extends RMAppEvent {
-  private String targetQueue;
-  private SettableFuture<Object> result;
-  
-  public RMAppMoveEvent(ApplicationId id, String newQueue,
-      SettableFuture<Object> resultFuture) {
-    super(id, RMAppEventType.MOVE);
-    this.targetQueue = newQueue;
-    this.result = resultFuture;
-  }
-  
-  public String getTargetQueue() {
-    return targetQueue;
-  }
-  
-  public SettableFuture<Object> getResult() {
-    return result;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 4818ea3..c0cc6b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -57,6 +57,8 @@ import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -64,7 +66,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMoveEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -360,6 +361,13 @@ public abstract class AbstractYarnScheduler
         + " does not support moving apps between queues");
   }
 
+  @Override
+  public void preValidateMoveApplication(ApplicationId appId,
+      String newQueue) throws YarnException {
+    throw new YarnException(getClass().getSimpleName()
+        + " does not support pre-validation of moving apps between queues");
+  }
+
   public void removeQueue(String queueName) throws YarnException {
     throw new YarnException(getClass().getSimpleName()
         + " does not support removing queues");
@@ -675,10 +683,10 @@ public abstract class AbstractYarnScheduler
         throw new YarnException(errMsg);
       }
       // generate move events for each pending/running app
-      for (ApplicationAttemptId app : apps) {
-        SettableFuture<Object> future = SettableFuture.create();
-        this.rmContext.getDispatcher().getEventHandler().handle(
-            new RMAppMoveEvent(app.getApplicationId(), destQueue, future));
+      for (ApplicationAttemptId appAttemptId : apps) {
+        this.rmContext.getDispatcher().getEventHandler()
+            .handle(new RMAppManagerEvent(appAttemptId.getApplicationId(),
+                destQueue, RMAppManagerEventType.APP_MOVE));
       }
     } finally {
       writeLock.unlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index 7167384..ea1ae60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -230,6 +230,17 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
       throws YarnException;
 
   /**
+   *
+   * @param appId Application ID
+   * @param newQueue Target QueueName
+   * @throws YarnException if the pre-validation for move cannot be carried out
+   */
+  @LimitedPrivate("yarn")
+  @Evolving
+  public void preValidateMoveApplication(ApplicationId appId,
+      String newQueue) throws YarnException;
+
+  /**
    * Completely drain sourceQueue of applications, by moving all of them to
    * destQueue.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index dd2f0d9..3372392 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -32,8 +32,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -67,6 +69,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import com.google.common.collect.Sets;
 
 public abstract class AbstractCSQueue implements CSQueue {
+
   private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);  
   volatile CSQueue parent;
   final String queueName;
@@ -837,4 +840,10 @@ public abstract class AbstractCSQueue implements CSQueue {
 
     return true;
   }
+
+  @Override
+  public void validateSubmitApplication(ApplicationId applicationId,
+      String userName, String queue) throws AccessControlException {
+    // Dummy implementation
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
index baf60e4..550e206 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
@@ -362,4 +362,14 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    * @return readLock of corresponding queue.
    */
   public ReentrantReadWriteLock.ReadLock getReadLock();
+
+  /**
+   * Validate submitApplication api so that moveApplication do a pre-check.
+   * @param applicationId Application ID
+   * @param userName User Name
+   * @param queue Queue Name
+   * @throws AccessControlException if any acl violation is there.
+   */
+  public void validateSubmitApplication(ApplicationId applicationId,
+      String userName, String queue) throws AccessControlException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index e42b20c..9a73a65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -2049,9 +2049,8 @@ public class CapacityScheduler extends
           sourceQueueName);
       String destQueueName = handleMoveToPlanQueue(targetQueueName);
       LeafQueue dest = this.queueManager.getAndCheckLeafQueue(destQueueName);
-      // Validation check - ACLs, submission limits for user & queue
+
       String user = app.getUser();
-      checkQueuePartition(app, dest);
       try {
         dest.submitApplication(appId, user, destQueueName);
       } catch (AccessControlException e) {
@@ -2079,6 +2078,30 @@ public class CapacityScheduler extends
     }
   }
 
+  @Override
+  public void preValidateMoveApplication(ApplicationId appId,
+      String newQueue) throws YarnException {
+    try {
+      writeLock.lock();
+      FiCaSchedulerApp app = getApplicationAttempt(
+          ApplicationAttemptId.newInstance(appId, 0));
+      String sourceQueueName = app.getQueue().getQueueName();
+      this.queueManager.getAndCheckLeafQueue(sourceQueueName);
+      String destQueueName = handleMoveToPlanQueue(newQueue);
+      LeafQueue dest = this.queueManager.getAndCheckLeafQueue(destQueueName);
+      // Validation check - ACLs, submission limits for user & queue
+      String user = app.getUser();
+      checkQueuePartition(app, dest);
+      try {
+        dest.validateSubmitApplication(appId, user, destQueueName);
+      } catch (AccessControlException e) {
+        throw new YarnException(e);
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
   /**
    * Check application can be moved to queue with labels enabled. All labels in
    * application life time will be checked

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 9661206..1c6471f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -564,6 +564,21 @@ public class LeafQueue extends AbstractCSQueue {
   public void submitApplication(ApplicationId applicationId, String userName,
       String queue)  throws AccessControlException {
     // Careful! Locking order is important!
+    validateSubmitApplication(applicationId, userName, queue);
+
+    // Inform the parent queue
+    try {
+      getParent().submitApplication(applicationId, userName, queue);
+    } catch (AccessControlException ace) {
+      LOG.info("Failed to submit application to parent-queue: " +
+          getParent().getQueuePath(), ace);
+      throw ace;
+    }
+
+  }
+
+  public void validateSubmitApplication(ApplicationId applicationId,
+      String userName, String queue) throws AccessControlException {
     try {
       writeLock.lock();
       // Check if the queue is accepting jobs
@@ -598,15 +613,13 @@ public class LeafQueue extends AbstractCSQueue {
       writeLock.unlock();
     }
 
-    // Inform the parent queue
     try {
-      getParent().submitApplication(applicationId, userName, queue);
+      getParent().validateSubmitApplication(applicationId, userName, queue);
     } catch (AccessControlException ace) {
       LOG.info("Failed to submit application to parent-queue: " + 
           getParent().getQueuePath(), ace);
       throw ace;
     }
-
   }
   
   public Resource getAMResourceLimit() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index fd0c68b..0ba4ede 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -340,16 +340,7 @@ public class ParentQueue extends AbstractCSQueue {
     try {
       writeLock.lock();
       // Sanity check
-      if (queue.equals(queueName)) {
-        throw new AccessControlException(
-            "Cannot submit application " + "to non-leaf queue: " + queueName);
-      }
-
-      if (state != QueueState.RUNNING) {
-        throw new AccessControlException("Queue " + getQueuePath()
-            + " is STOPPED. Cannot accept submission of application: "
-            + applicationId);
-      }
+      validateSubmitApplication(applicationId, user, queue);
 
       addApplication(applicationId, user);
     } finally {
@@ -369,6 +360,24 @@ public class ParentQueue extends AbstractCSQueue {
     }
   }
 
+  public void validateSubmitApplication(ApplicationId applicationId,
+      String userName, String queue) throws AccessControlException {
+    try {
+      writeLock.lock();
+      if (queue.equals(queueName)) {
+        throw new AccessControlException(
+            "Cannot submit application " + "to non-leaf queue: " + queueName);
+      }
+
+      if (state != QueueState.RUNNING) {
+        throw new AccessControlException("Queue " + getQueuePath()
+            + " is STOPPED. Cannot accept submission of application: "
+            + applicationId);
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
 
   @Override
   public void submitApplicationAttempt(FiCaSchedulerApp application,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index fbcac76..03df5d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -1544,7 +1544,41 @@ public class FairScheduler extends
       writeLock.unlock();
     }
   }
-  
+
+  @Override
+  public void preValidateMoveApplication(ApplicationId appId, String newQueue)
+      throws YarnException {
+    try {
+      writeLock.lock();
+      SchedulerApplication<FSAppAttempt> app = applications.get(appId);
+      if (app == null) {
+        throw new YarnException("App to be moved " + appId + " not found.");
+      }
+
+      FSAppAttempt attempt = app.getCurrentAppAttempt();
+      // To serialize with FairScheduler#allocate, synchronize on app attempt
+
+      try {
+        attempt.getWriteLock().lock();
+        FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
+        String destQueueName = handleMoveToPlanQueue(newQueue);
+        FSLeafQueue targetQueue = queueMgr.getLeafQueue(destQueueName, false);
+        if (targetQueue == null) {
+          throw new YarnException("Target queue " + newQueue
+              + " not found or is not a leaf queue.");
+        }
+
+        if (oldQueue.isRunnableApp(attempt)) {
+          verifyMoveDoesNotViolateConstraints(attempt, oldQueue, targetQueue);
+        }
+      } finally {
+        attempt.getWriteLock().unlock();
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
   private void verifyMoveDoesNotViolateConstraints(FSAppAttempt app,
       FSLeafQueue oldQueue, FSLeafQueue targetQueue) throws YarnException {
     String queueName = targetQueue.getQueueName();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
index d2bde80..05b25df 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
@@ -87,10 +87,10 @@ public class TestMoveApplication {
               application.getApplicationId(), "newqueue"));
       fail("Should have hit exception");
     } catch (YarnException ex) {
-      assertEquals("Move not supported", ex.getCause().getMessage());
+      assertEquals("Move not supported", ex.getMessage());
     }
   }
-  
+
   @Test (timeout = 10000)
   public void testMoveTooLate() throws Exception {
     // Submit application
@@ -178,5 +178,13 @@ public class TestMoveApplication {
         QueueACL acl, String queueName) {
       return acl != QueueACL.ADMINISTER_QUEUE;
     }
+
+    @Override
+    public void preValidateMoveApplication(ApplicationId appId, String newQueue)
+        throws YarnException {
+      if (failMove) {
+        throw new YarnException("Move not supported");
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563480dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
index 439e9df..0a864fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
@@ -457,6 +457,7 @@ public class TestCapacitySchedulerNodeLabelUpdate {
     CapacityScheduler scheduler =
         ((CapacityScheduler) rm.getResourceScheduler());
     try {
+      scheduler.preValidateMoveApplication(app1.getApplicationId(), "a2");
       scheduler.moveApplication(app1.getApplicationId(), "a2");
       fail("Should throw exception since target queue doesnt have "
           + "required labels");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[49/50] [abbrv] hadoop git commit: HADOOP-13852 hadoop build to allow hadoop version property to be explicitly set. Contriibuted by Steve Loughran

Posted by st...@apache.org.
HADOOP-13852 hadoop build to allow hadoop version property to be explicitly set. Contriibuted by Steve Loughran


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

Branch: refs/heads/HADOOP-13345
Commit: c2655157257079b8541d71bb1e5b6cbae75561ff
Parents: 0ef7961
Author: Steve Loughran <st...@apache.org>
Authored: Thu Dec 8 17:57:15 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Dec 8 17:57:15 2016 +0000

----------------------------------------------------------------------
 BUILDING.txt                                             | 11 ++++++++++-
 hadoop-common-project/hadoop-common/pom.xml              |  3 +++
 .../src/main/resources/common-version-info.properties    |  4 ++--
 .../src/main/resources/yarn-version-info.properties      |  2 +-
 4 files changed, 16 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2655157/BUILDING.txt
----------------------------------------------------------------------
diff --git a/BUILDING.txt b/BUILDING.txt
index 8b2bba6..7afc3f0 100644
--- a/BUILDING.txt
+++ b/BUILDING.txt
@@ -390,7 +390,7 @@ http://www.zlib.net/
 ----------------------------------------------------------------------------------
 Building distributions:
 
- * Build distribution with native code    : mvn package [-Pdist][-Pdocs][-Psrc][-Dtar]
+ * Build distribution with native code    : mvn package [-Pdist][-Pdocs][-Psrc][-Dtar][-Dmaven.javadoc.skip=true]
 
 ----------------------------------------------------------------------------------
 Running compatibility checks with checkcompatibility.py
@@ -402,3 +402,12 @@ managers to compare the compatibility of a previous and current release.
 As an example, this invocation will check the compatibility of interfaces annotated as Public or LimitedPrivate:
 
 ./dev-support/bin/checkcompatibility.py --annotation org.apache.hadoop.classification.InterfaceAudience.Public --annotation org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate --include "hadoop.*" branch-2.7.2 trunk
+
+----------------------------------------------------------------------------------
+Changing the Hadoop version declared returned by VersionInfo
+
+If for compatibility reasons the version of Hadoop has to be declared as a 2.x release in the information returned by
+org.apache.hadoop.util.VersionInfo, set the property declared.hadoop.version to the desired version.
+For example: mvn package -Pdist -Ddeclared.hadoop.version=2.11
+
+If unset, the project version declared in the POM file is used.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2655157/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index c9b282f..aa20f79 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -36,6 +36,9 @@
     <is.hadoop.common.component>true</is.hadoop.common.component>
     <wsce.config.dir>../etc/hadoop</wsce.config.dir>
     <wsce.config.file>wsce-site.xml</wsce.config.file>
+    <!-- the version of Hadoop declared in the version resources; can be overridden
+    so that Hadoop 3.x can declare itself a 2.x artifact. -->
+    <declared.hadoop.version>${pom.version}</declared.hadoop.version>
   </properties>
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2655157/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties b/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties
index ad9a24d..9b74960 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties
+++ b/hadoop-common-project/hadoop-common/src/main/resources/common-version-info.properties
@@ -16,11 +16,11 @@
 # limitations under the License.
 #
 
-version=${pom.version}
+version=${declared.hadoop.version}
 revision=${version-info.scm.commit}
 branch=${version-info.scm.branch}
 user=${user.name}
 date=${version-info.build.time}
 url=${version-info.scm.uri}
 srcChecksum=${version-info.source.md5}
-protocVersion=${protobuf.version}
\ No newline at end of file
+protocVersion=${protobuf.version}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2655157/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-version-info.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-version-info.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-version-info.properties
index 9a8575c..ee6f13d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-version-info.properties
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-version-info.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 #
 
-version=${pom.version}
+version=${declared.hadoop.version}
 revision=${version-info.scm.commit}
 branch=${version-info.scm.branch}
 user=${user.name}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/50] [abbrv] hadoop git commit: YARN-5136. Error in handling event type APP_ATTEMPT_REMOVED to the scheduler (Contributed by Wilfred Spiegelenburg via Daniel Templeton)

Posted by st...@apache.org.
YARN-5136. Error in handling event type APP_ATTEMPT_REMOVED to the scheduler
(Contributed by Wilfred Spiegelenburg via Daniel Templeton)


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

Branch: refs/heads/HADOOP-13345
Commit: 9f5d2c4fff6d31acc8b422b52462ef4927c4eea1
Parents: ab923a5
Author: Daniel Templeton <te...@apache.org>
Authored: Wed Dec 7 11:12:14 2016 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Wed Dec 7 11:12:14 2016 -0800

----------------------------------------------------------------------
 .../scheduler/fair/FairScheduler.java           | 32 +++++--
 .../scheduler/fair/TestFairScheduler.java       | 89 ++++++++++++++++++++
 2 files changed, 115 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f5d2c4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 03df5d4..e790bc2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -633,8 +633,7 @@ public class FairScheduler extends
       RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
     try {
       writeLock.lock();
-      LOG.info(
-          "Application " + applicationAttemptId + " is done." + " finalState="
+      LOG.info("Application " + applicationAttemptId + " is done. finalState="
               + rmAppAttemptFinalState);
       FSAppAttempt attempt = getApplicationAttempt(applicationAttemptId);
 
@@ -644,6 +643,13 @@ public class FairScheduler extends
         return;
       }
 
+      // Check if the attempt is already stopped and don't stop it twice.
+      if (attempt.isStopped()) {
+        LOG.info("Application " + applicationAttemptId + " has already been "
+            + "stopped!");
+        return;
+      }
+
       // Release all the running containers
       for (RMContainer rmContainer : attempt.getLiveContainers()) {
         if (keepContainers && rmContainer.getState().equals(
@@ -1521,6 +1527,13 @@ public class FairScheduler extends
       try {
         attempt.getWriteLock().lock();
         FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
+        // Check if the attempt is already stopped: don't move stopped app
+        // attempt. The attempt has already been removed from all queues.
+        if (attempt.isStopped()) {
+          LOG.info("Application " + appId + " is stopped and can't be moved!");
+          throw new YarnException("Application " + appId
+              + " is stopped and can't be moved!");
+        }
         String destQueueName = handleMoveToPlanQueue(queueName);
         FSLeafQueue targetQueue = queueMgr.getLeafQueue(destQueueName, false);
         if (targetQueue == null) {
@@ -1617,16 +1630,23 @@ public class FairScheduler extends
    * operations will be atomic.
    */
   private void executeMove(SchedulerApplication<FSAppAttempt> app,
-      FSAppAttempt attempt, FSLeafQueue oldQueue, FSLeafQueue newQueue) {
-    boolean wasRunnable = oldQueue.removeApp(attempt);
+      FSAppAttempt attempt, FSLeafQueue oldQueue, FSLeafQueue newQueue)
+      throws YarnException {
+    // Check current runs state. Do not remove the attempt from the queue until
+    // after the check has been performed otherwise it could remove the app
+    // from a queue without moving it to a new queue.
+    boolean wasRunnable = oldQueue.isRunnableApp(attempt);
     // if app was not runnable before, it may be runnable now
     boolean nowRunnable = maxRunningEnforcer.canAppBeRunnable(newQueue,
         attempt);
     if (wasRunnable && !nowRunnable) {
-      throw new IllegalStateException("Should have already verified that app "
+      throw new YarnException("Should have already verified that app "
           + attempt.getApplicationId() + " would be runnable in new queue");
     }
-    
+
+    // Now it is safe to remove from the queue.
+    oldQueue.removeApp(attempt);
+
     if (wasRunnable) {
       maxRunningEnforcer.untrackRunnableApp(attempt);
     } else if (nowRunnable) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f5d2c4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 55f8849..5aa1e2d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -4552,6 +4552,95 @@ public class TestFairScheduler extends FairSchedulerTestBase {
   }
 
   @Test
+  public void testDoubleRemoval() throws Exception {
+    String testUser = "user1"; // convenience var
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    ApplicationAttemptId attemptId = createAppAttemptId(1, 1);
+    // The placement rule will add the app to the user based queue but the
+    // passed in queue must exist.
+    AppAddedSchedulerEvent appAddedEvent =
+        new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser,
+            testUser);
+    scheduler.handle(appAddedEvent);
+    AppAttemptAddedSchedulerEvent attemptAddedEvent =
+        new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
+    scheduler.handle(attemptAddedEvent);
+
+    // Get a handle on the attempt.
+    FSAppAttempt attempt = scheduler.getSchedulerApp(attemptId);
+
+    AppAttemptRemovedSchedulerEvent attemptRemovedEvent =
+        new AppAttemptRemovedSchedulerEvent(createAppAttemptId(1, 1),
+            RMAppAttemptState.FINISHED, false);
+
+    // Make sure the app attempt is in the queue.
+    List<ApplicationAttemptId> attemptList =
+        scheduler.getAppsInQueue(testUser);
+    assertNotNull("Queue missing", attemptList);
+    assertTrue("Attempt should be in the queue",
+        attemptList.contains(attemptId));
+    assertFalse("Attempt is stopped", attempt.isStopped());
+
+    // Now remove the app attempt
+    scheduler.handle(attemptRemovedEvent);
+    // The attempt is not in the queue, and stopped
+    attemptList = scheduler.getAppsInQueue(testUser);
+    assertFalse("Attempt should not be in the queue",
+        attemptList.contains(attemptId));
+    assertTrue("Attempt should have been stopped", attempt.isStopped());
+
+    // Now remove the app attempt again, since it is stopped nothing happens.
+    scheduler.handle(attemptRemovedEvent);
+    // The attempt should still show the original queue info.
+    assertTrue("Attempt queue has changed",
+        attempt.getQueue().getName().endsWith(testUser));
+  }
+
+  @Test (expected = YarnException.class)
+  public void testMoveAfterRemoval() throws Exception {
+    String testUser = "user1"; // convenience var
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    ApplicationAttemptId attemptId = createAppAttemptId(1, 1);
+    AppAddedSchedulerEvent appAddedEvent =
+        new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser,
+            testUser);
+    scheduler.handle(appAddedEvent);
+    AppAttemptAddedSchedulerEvent attemptAddedEvent =
+        new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
+    scheduler.handle(attemptAddedEvent);
+
+    // Get a handle on the attempt.
+    FSAppAttempt attempt = scheduler.getSchedulerApp(attemptId);
+
+    AppAttemptRemovedSchedulerEvent attemptRemovedEvent =
+        new AppAttemptRemovedSchedulerEvent(createAppAttemptId(1, 1),
+            RMAppAttemptState.FINISHED, false);
+
+    // Remove the app attempt
+    scheduler.handle(attemptRemovedEvent);
+    // Make sure the app attempt is not in the queue and stopped.
+    List<ApplicationAttemptId> attemptList =
+        scheduler.getAppsInQueue(testUser);
+    assertNotNull("Queue missing", attemptList);
+    assertFalse("Attempt should not be in the queue",
+        attemptList.contains(attemptId));
+    assertTrue("Attempt should have been stopped", attempt.isStopped());
+    // The attempt should still show the original queue info.
+    assertTrue("Attempt queue has changed",
+        attempt.getQueue().getName().endsWith(testUser));
+
+    // Now move the app: not using an event since there is none
+    // in the scheduler. This should throw.
+    scheduler.moveApplication(attemptId.getApplicationId(), "default");
+  }
+
+  @Test
   public void testPerfMetricsInited() {
     scheduler.init(conf);
     scheduler.start();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/50] [abbrv] hadoop git commit: HDFS-10206. Datanodes not sorted properly by distance when the reader isn't a datanode. (Nandakumar via mingma)

Posted by st...@apache.org.
HDFS-10206. Datanodes not sorted properly by distance when the reader isn't a datanode. (Nandakumar via mingma)


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

Branch: refs/heads/HADOOP-13345
Commit: c73e08a6dad46cad14b38a4a586a5cda1622b206
Parents: 563480d
Author: Ming Ma <mi...@apache.org>
Authored: Wed Dec 7 08:26:09 2016 -0800
Committer: Ming Ma <mi...@apache.org>
Committed: Wed Dec 7 08:26:09 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/net/NetworkTopology.java  | 158 +++++++++++++++++--
 .../server/blockmanagement/DatanodeManager.java |  12 +-
 .../apache/hadoop/net/TestNetworkTopology.java  |  29 +++-
 3 files changed, 182 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73e08a6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
index 14c870d..5751d2b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
@@ -57,6 +57,10 @@ public class NetworkTopology {
   public static final Logger LOG =
       LoggerFactory.getLogger(NetworkTopology.class);
 
+  private static final char PATH_SEPARATOR = '/';
+  private static final String PATH_SEPARATOR_STR = "/";
+  private static final String ROOT = "/";
+
   public static class InvalidTopologyException extends RuntimeException {
     private static final long serialVersionUID = 1L;
     public InvalidTopologyException(String msg) {
@@ -916,7 +920,7 @@ public class NetworkTopology {
     }
   }
 
-  /** convert a network tree to a string */
+  /** convert a network tree to a string. */
   @Override
   public String toString() {
     // print the number of racks
@@ -970,19 +974,108 @@ public class NetworkTopology {
    * @return weight
    */
   protected int getWeight(Node reader, Node node) {
-    // 0 is local, 1 is same rack, 2 is off rack
-    // Start off by initializing to off rack
-    int weight = 2;
-    if (reader != null) {
-      if (reader.equals(node)) {
-        weight = 0;
-      } else if (isOnSameRack(reader, node)) {
-        weight = 1;
+    // 0 is local, 2 is same rack, and each level on each node increases the
+    //weight by 1
+    //Start off by initializing to Integer.MAX_VALUE
+    int weight = Integer.MAX_VALUE;
+    if (reader != null && node != null) {
+      if(reader.equals(node)) {
+        return 0;
+      }
+      int maxReaderLevel = reader.getLevel();
+      int maxNodeLevel = node.getLevel();
+      int currentLevelToCompare = maxReaderLevel > maxNodeLevel ?
+          maxNodeLevel : maxReaderLevel;
+      Node r = reader;
+      Node n = node;
+      weight = 0;
+      while(r != null && r.getLevel() > currentLevelToCompare) {
+        r = r.getParent();
+        weight++;
+      }
+      while(n != null && n.getLevel() > currentLevelToCompare) {
+        n = n.getParent();
+        weight++;
+      }
+      while(r != null && n != null && !r.equals(n)) {
+        r = r.getParent();
+        n = n.getParent();
+        weight+=2;
+      }
+    }
+    return weight;
+  }
+
+  /**
+   * Returns an integer weight which specifies how far away <i>node</i> is
+   * from <i>reader</i>. A lower value signifies that a node is closer.
+   * It uses network location to calculate the weight
+   *
+   * @param reader Node where data will be read
+   * @param node Replica of data
+   * @return weight
+   */
+  private static int getWeightUsingNetworkLocation(Node reader, Node node) {
+    //Start off by initializing to Integer.MAX_VALUE
+    int weight = Integer.MAX_VALUE;
+    if(reader != null && node != null) {
+      String readerPath = normalizeNetworkLocationPath(
+          reader.getNetworkLocation());
+      String nodePath = normalizeNetworkLocationPath(
+          node.getNetworkLocation());
+
+      //same rack
+      if(readerPath.equals(nodePath)) {
+        if(reader.getName().equals(node.getName())) {
+          weight = 0;
+        } else {
+          weight = 2;
+        }
+      } else {
+        String[] readerPathToken = readerPath.split(PATH_SEPARATOR_STR);
+        String[] nodePathToken = nodePath.split(PATH_SEPARATOR_STR);
+        int maxLevelToCompare = readerPathToken.length > nodePathToken.length ?
+            nodePathToken.length : readerPathToken.length;
+        int currentLevel = 1;
+        //traverse through the path and calculate the distance
+        while(currentLevel < maxLevelToCompare) {
+          if(!readerPathToken[currentLevel]
+              .equals(nodePathToken[currentLevel])){
+            break;
+          }
+          currentLevel++;
+        }
+        weight = (readerPathToken.length - currentLevel) +
+            (nodePathToken.length - currentLevel);
       }
     }
     return weight;
   }
 
+  /** Normalize a path by stripping off any trailing {@link #PATH_SEPARATOR}.
+   * @param path path to normalize.
+   * @return the normalised path
+   * If <i>path</i>is null or empty {@link #ROOT} is returned
+   * @throws IllegalArgumentException if the first character of a non empty path
+   * is not {@link #PATH_SEPARATOR}
+   */
+  private static String normalizeNetworkLocationPath(String path) {
+    if (path == null || path.length() == 0) {
+      return ROOT;
+    }
+
+    if (path.charAt(0) != PATH_SEPARATOR) {
+      throw new IllegalArgumentException("Network Location"
+          + "path doesn't start with " +PATH_SEPARATOR+ ": "+path);
+    }
+
+    int len = path.length();
+    if (path.charAt(len-1) == PATH_SEPARATOR) {
+      return path.substring(0, len-1);
+    }
+    return path;
+  }
+
   /**
    * Sort nodes array by network distance to <i>reader</i>.
    * <p/>
@@ -999,10 +1092,55 @@ public class NetworkTopology {
    * @param activeLen Number of active nodes at the front of the array
    */
   public void sortByDistance(Node reader, Node[] nodes, int activeLen) {
+    /*
+     * This method is called if the reader is a datanode,
+     * so nonDataNodeReader flag is set to false.
+     */
+    sortByDistance(reader, nodes, activeLen, false);
+  }
+
+  /**
+   * Sort nodes array by network distance to <i>reader</i>.
+   * <p/> using network location. This is used when the reader
+   * is not a datanode. Sorting the nodes based on network distance
+   * from the reader reduces network traffic and improves
+   * performance.
+   * <p/>
+   *
+   * @param reader    Node where data will be read
+   * @param nodes     Available replicas with the requested data
+   * @param activeLen Number of active nodes at the front of the array
+   */
+  public void sortByDistanceUsingNetworkLocation(Node reader, Node[] nodes,
+      int activeLen) {
+    /*
+     * This method is called if the reader is not a datanode,
+     * so nonDataNodeReader flag is set to true.
+     */
+    sortByDistance(reader, nodes, activeLen, true);
+  }
+
+  /**
+   * Sort nodes array by network distance to <i>reader</i>.
+   * <p/>
+   * As an additional twist, we also randomize the nodes at each network
+   * distance. This helps with load balancing when there is data skew.
+   *
+   * @param reader    Node where data will be read
+   * @param nodes     Available replicas with the requested data
+   * @param activeLen Number of active nodes at the front of the array
+   * @param nonDataNodeReader True if the reader is not a datanode
+   */
+  private void sortByDistance(Node reader, Node[] nodes, int activeLen,
+      boolean nonDataNodeReader) {
     /** Sort weights for the nodes array */
     int[] weights = new int[activeLen];
     for (int i=0; i<activeLen; i++) {
-      weights[i] = getWeight(reader, nodes[i]);
+      if(nonDataNodeReader) {
+        weights[i] = getWeightUsingNetworkLocation(reader, nodes[i]);
+      } else {
+        weights[i] = getWeight(reader, nodes[i]);
+      }
     }
     // Add weight/node pairs to a TreeMap to sort
     TreeMap<Integer, List<Node>> tree = new TreeMap<Integer, List<Node>>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73e08a6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 47f15c4..6477d5c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -443,9 +443,11 @@ public class DatanodeManager {
       Comparator<DatanodeInfo> comparator) {
     // As it is possible for the separation of node manager and datanode, 
     // here we should get node but not datanode only .
+    boolean nonDatanodeReader = false;
     Node client = getDatanodeByHost(targetHost);
     if (client == null) {
-      List<String> hosts = new ArrayList<> (1);
+      nonDatanodeReader = true;
+      List<String> hosts = new ArrayList<>(1);
       hosts.add(targetHost);
       List<String> resolvedHosts = dnsToSwitchMapping.resolve(hosts);
       if (resolvedHosts != null && !resolvedHosts.isEmpty()) {
@@ -470,8 +472,12 @@ public class DatanodeManager {
       --lastActiveIndex;
     }
     int activeLen = lastActiveIndex + 1;
-    networktopology.sortByDistance(client, lb.getLocations(), activeLen);
-
+    if(nonDatanodeReader) {
+      networktopology.sortByDistanceUsingNetworkLocation(client,
+          lb.getLocations(), activeLen);
+    } else {
+      networktopology.sortByDistance(client, lb.getLocations(), activeLen);
+    }
     // must update cache since we modified locations array
     lb.updateCachedStorageInfo();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73e08a6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
index d149f65..3a281fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
@@ -220,11 +220,9 @@ public class TestNetworkTopology {
     testNodes[2] = dataNodes[3];
     cluster.setRandomSeed(0xDEAD);
     cluster.sortByDistance(dataNodes[0], testNodes, testNodes.length);
-    // sortByDistance does not take the "data center" layer into consideration
-    // and it doesn't sort by getDistance, so 1, 5, 3 is also valid here
     assertTrue(testNodes[0] == dataNodes[1]);
-    assertTrue(testNodes[1] == dataNodes[5]);
-    assertTrue(testNodes[2] == dataNodes[3]);
+    assertTrue(testNodes[1] == dataNodes[3]);
+    assertTrue(testNodes[2] == dataNodes[5]);
 
     // Array of just rack-local nodes
     // Expect a random first node
@@ -264,6 +262,29 @@ public class TestNetworkTopology {
       }
     }
     assertTrue("Expected to find a different first location", foundRandom);
+
+    //Reader is not a datanode, but is in one of the datanode's rack.
+    testNodes[0] = dataNodes[0];
+    testNodes[1] = dataNodes[5];
+    testNodes[2] = dataNodes[8];
+    Node rackClient = new NodeBase("/d3/r1/25.25.25");
+    cluster.setRandomSeed(0xDEADBEEF);
+    cluster.sortByDistance(rackClient, testNodes, testNodes.length);
+    assertTrue(testNodes[0] == dataNodes[8]);
+    assertTrue(testNodes[1] == dataNodes[5]);
+    assertTrue(testNodes[2] == dataNodes[0]);
+
+    //Reader is not a datanode , but is in one of the datanode's data center.
+    testNodes[0] = dataNodes[8];
+    testNodes[1] = dataNodes[5];
+    testNodes[2] = dataNodes[0];
+    Node dcClient = new NodeBase("/d1/r2/25.25.25");
+    cluster.setRandomSeed(0xDEADBEEF);
+    cluster.sortByDistance(dcClient, testNodes, testNodes.length);
+    assertTrue(testNodes[0] == dataNodes[0]);
+    assertTrue(testNodes[1] == dataNodes[5]);
+    assertTrue(testNodes[2] == dataNodes[8]);
+
   }
   
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/50] [abbrv] hadoop git commit: HDFS-11198. NN UI should link DN web address using hostnames. Contributed by Weiwei Yang.

Posted by st...@apache.org.
HDFS-11198. NN UI should link DN web address using hostnames. Contributed by  Weiwei Yang.


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

Branch: refs/heads/HADOOP-13345
Commit: 4dd4f3a4bb483795e8669ec34d3efcd6c1b8b465
Parents: 7b988e8
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue Dec 6 08:06:50 2016 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Tue Dec 6 08:06:50 2016 -0600

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html    |  8 +-------
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js      | 12 ++++++------
 2 files changed, 7 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4dd4f3a4/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
index 13569fe..3598c80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
@@ -314,13 +314,7 @@
   {#LiveNodes}
   <tr>
     <td ng-value="{state}-{name}" class="dfshealth-node-icon dfshealth-node-{state}">{name} ({xferaddr})</td>
-    <td ng-value="{state}-{name}">
-    {@select key=secureMode}
-      {@eq value="off"}<a href='//{infoAddr}'>{infoAddr}</a>{/eq}
-      {@eq value="on"}<a href='//{infoSecureAddr}'>{infoSecureAddr}</a>{/eq}
-      {@default}<a href='//{infoAddr}'>{infoAddr}</a>{/default}
-    {/select}
-    </td>
+    <td ng-value="{state}-{name}"><a href='//{dnWebAddress}'>{dnWebAddress}</a></td>
     <td ng-value="{lastContact}">{lastContact}s</td>
     <td ng-value="{usedPercentage}">
       <div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4dd4f3a4/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
index f230b0f..02aa895 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
@@ -212,12 +212,12 @@
           var n = nodes[i];
           n.usedPercentage = Math.round((n.used + n.nonDfsUsedSpace) * 1.0 / n.capacity * 100);
 
-          var addr = n.infoSecureAddr;
-          var position = addr.lastIndexOf(":");
-          var port = addr.substring(position + 1, addr.length);
-          n.secureMode = "off";
-          if (port != 0) {
-            n.secureMode = "on";
+          var port = n.infoAddr.split(":")[1];
+          var securePort = n.infoSecureAddr.split(":")[1];
+          var dnHost = n.name.split(":")[0];
+          n.dnWebAddress = dnHost + ":" + port;
+          if (securePort != 0) {
+            n.dnWebAddress = dnHost + ":" + securePort;
           }
 
           if (n.adminState === "In Service") {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/50] [abbrv] hadoop git commit: HADOOP-13827. Add reencryptEncryptedKey interface to KMS.

Posted by st...@apache.org.
HADOOP-13827. Add reencryptEncryptedKey interface to KMS.


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

Branch: refs/heads/HADOOP-13345
Commit: 79d90b810c14d5e3abab75235f587663834ce36c
Parents: df983b5
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Dec 6 12:04:04 2016 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Dec 6 12:04:04 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/crypto/key/KeyProvider.java   |  28 +++++
 .../crypto/key/KeyProviderCryptoExtension.java  |  89 +++++++++++++--
 .../crypto/key/kms/KMSClientProvider.java       |  77 +++++++++----
 .../hadoop/crypto/key/kms/KMSRESTConstants.java |   1 +
 .../key/kms/LoadBalancingKMSClientProvider.java |  18 +++
 .../key/TestKeyProviderCryptoExtension.java     | 114 +++++++++++++++++++
 ...rKeyGeneratorKeyProviderCryptoExtension.java |   7 +-
 .../hadoop/crypto/key/kms/server/KMS.java       |  38 +++++--
 .../hadoop/crypto/key/kms/server/KMSAudit.java  |   4 +-
 .../kms/server/KeyAuthorizationKeyProvider.java |  13 +++
 .../hadoop-kms/src/site/markdown/index.md.vm    |  39 ++++++-
 .../hadoop/crypto/key/kms/server/TestKMS.java   |  90 +++++++++++++--
 .../crypto/key/kms/server/TestKMSAudit.java     |  11 +-
 13 files changed, 469 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
index c99a7bf..d54c18e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
@@ -33,6 +33,8 @@ import java.util.Map;
 
 import com.google.gson.stream.JsonReader;
 import com.google.gson.stream.JsonWriter;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -86,6 +88,7 @@ public abstract class KeyProvider {
       return material;
     }
 
+    @Override
     public String toString() {
       StringBuilder buf = new StringBuilder();
       buf.append("key(");
@@ -105,6 +108,31 @@ public abstract class KeyProvider {
       }
       return buf.toString();
     }
+
+    @Override
+    public boolean equals(Object rhs) {
+      if (this == rhs) {
+        return true;
+      }
+      if (rhs == null || getClass() != rhs.getClass()) {
+        return false;
+      }
+      final KeyVersion kv = (KeyVersion) rhs;
+      return new EqualsBuilder().
+          append(name, kv.name).
+          append(versionName, kv.versionName).
+          append(material, kv.material).
+          isEquals();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().
+          append(name).
+          append(versionName).
+          append(material).
+          toHashCode();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 0543222..9ae98b4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -188,8 +188,8 @@ public class KeyProviderCryptoExtension extends
     public void drain(String keyName);
 
     /**
-     * Generates a key material and encrypts it using the given key version name
-     * and initialization vector. The generated key material is of the same
+     * Generates a key material and encrypts it using the given key name.
+     * The generated key material is of the same
      * length as the <code>KeyVersion</code> material of the latest key version
      * of the key and is encrypted using the same cipher.
      * <p/>
@@ -210,7 +210,7 @@ public class KeyProviderCryptoExtension extends
         GeneralSecurityException;
 
     /**
-     * Decrypts an encrypted byte[] key material using the given a key version
+     * Decrypts an encrypted byte[] key material using the given key version
      * name and initialization vector.
      * 
      * @param encryptedKeyVersion
@@ -227,6 +227,26 @@ public class KeyProviderCryptoExtension extends
     public KeyVersion decryptEncryptedKey(
         EncryptedKeyVersion encryptedKeyVersion) throws IOException,
         GeneralSecurityException;
+
+    /**
+     * Re-encrypts an encrypted key version, using its initialization vector
+     * and key material, but with the latest key version name of its key name
+     * in the key provider.
+     * <p>
+     * If the latest key version name in the provider is the
+     * same as the one encrypted the passed-in encrypted key version, the same
+     * encrypted key version is returned.
+     * <p>
+     * NOTE: The generated key is not stored by the <code>KeyProvider</code>
+     *
+     * @param  ekv The EncryptedKeyVersion containing keyVersionName and IV.
+     * @return     The re-encrypted EncryptedKeyVersion.
+     * @throws IOException If the key material could not be re-encrypted.
+     * @throws GeneralSecurityException If the key material could not be
+     *                            re-encrypted because of a cryptographic issue.
+     */
+    EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+        throws IOException, GeneralSecurityException;
   }
 
   private static class DefaultCryptoExtension implements CryptoExtension {
@@ -258,25 +278,56 @@ public class KeyProviderCryptoExtension extends
       cc.generateSecureRandom(newKey);
       final byte[] iv = new byte[cc.getCipherSuite().getAlgorithmBlockSize()];
       cc.generateSecureRandom(iv);
+      Encryptor encryptor = cc.createEncryptor();
+      return generateEncryptedKey(encryptor, encryptionKey, newKey, iv);
+    }
+
+    private EncryptedKeyVersion generateEncryptedKey(final Encryptor encryptor,
+        final KeyVersion encryptionKey, final byte[] key, final byte[] iv)
+        throws IOException, GeneralSecurityException {
       // Encryption key IV is derived from new key's IV
       final byte[] encryptionIV = EncryptedKeyVersion.deriveIV(iv);
-      Encryptor encryptor = cc.createEncryptor();
       encryptor.init(encryptionKey.getMaterial(), encryptionIV);
-      int keyLen = newKey.length;
+      final int keyLen = key.length;
       ByteBuffer bbIn = ByteBuffer.allocateDirect(keyLen);
       ByteBuffer bbOut = ByteBuffer.allocateDirect(keyLen);
-      bbIn.put(newKey);
+      bbIn.put(key);
       bbIn.flip();
       encryptor.encrypt(bbIn, bbOut);
       bbOut.flip();
       byte[] encryptedKey = new byte[keyLen];
-      bbOut.get(encryptedKey);    
-      return new EncryptedKeyVersion(encryptionKeyName,
+      bbOut.get(encryptedKey);
+      return new EncryptedKeyVersion(encryptionKey.getName(),
           encryptionKey.getVersionName(), iv,
           new KeyVersion(encryptionKey.getName(), EEK, encryptedKey));
     }
 
     @Override
+    public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+        throws IOException, GeneralSecurityException {
+      final String ekName = ekv.getEncryptionKeyName();
+      final KeyVersion ekNow = keyProvider.getCurrentKey(ekName);
+      Preconditions
+          .checkNotNull(ekNow, "KeyVersion name '%s' does not exist", ekName);
+      Preconditions.checkArgument(ekv.getEncryptedKeyVersion().getVersionName()
+              .equals(KeyProviderCryptoExtension.EEK),
+          "encryptedKey version name must be '%s', is '%s'",
+          KeyProviderCryptoExtension.EEK,
+          ekv.getEncryptedKeyVersion().getVersionName());
+
+      if (ekv.getEncryptedKeyVersion().equals(ekNow)) {
+        // no-op if same key version
+        return ekv;
+      }
+
+      final KeyVersion dek = decryptEncryptedKey(ekv);
+      final CryptoCodec cc = CryptoCodec.getInstance(keyProvider.getConf());
+      final Encryptor encryptor = cc.createEncryptor();
+      return generateEncryptedKey(encryptor, ekNow, dek.getMaterial(),
+          ekv.getEncryptedKeyIv());
+    }
+
+    @Override
     public KeyVersion decryptEncryptedKey(
         EncryptedKeyVersion encryptedKeyVersion) throws IOException,
         GeneralSecurityException {
@@ -389,6 +440,28 @@ public class KeyProviderCryptoExtension extends
   }
 
   /**
+   * Re-encrypts an encrypted key version, using its initialization vector
+   * and key material, but with the latest key version name of its key name
+   * in the key provider.
+   * <p>
+   * If the latest key version name in the provider is the
+   * same as the one encrypted the passed-in encrypted key version, the same
+   * encrypted key version is returned.
+   * <p>
+   * NOTE: The generated key is not stored by the <code>KeyProvider</code>
+   *
+   * @param  ekv The EncryptedKeyVersion containing keyVersionName and IV.
+   * @return     The re-encrypted EncryptedKeyVersion.
+   * @throws IOException If the key material could not be re-encrypted
+   * @throws GeneralSecurityException If the key material could not be
+   *                            re-encrypted because of a cryptographic issue.
+   */
+  public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+      throws IOException, GeneralSecurityException {
+    return getExtension().reencryptEncryptedKey(ekv);
+  }
+
+  /**
    * Creates a <code>KeyProviderCryptoExtension</code> using a given
    * {@link KeyProvider}.
    * <p/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
index ff03db9..df6768d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
@@ -146,7 +146,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       List response = call(conn, null,
           HttpURLConnection.HTTP_OK, List.class);
       List<EncryptedKeyVersion> ekvs =
-          parseJSONEncKeyVersion(keyName, response);
+          parseJSONEncKeyVersions(keyName, response);
       keyQueue.addAll(ekvs);
     }
   }
@@ -221,37 +221,41 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
 
   @SuppressWarnings("rawtypes")
   private static List<EncryptedKeyVersion>
-      parseJSONEncKeyVersion(String keyName, List valueList) {
+      parseJSONEncKeyVersions(String keyName, List valueList) {
     List<EncryptedKeyVersion> ekvs = new LinkedList<EncryptedKeyVersion>();
     if (!valueList.isEmpty()) {
       for (Object values : valueList) {
         Map valueMap = (Map) values;
+        ekvs.add(parseJSONEncKeyVersion(keyName, valueMap));
+      }
+    }
+    return ekvs;
+  }
 
-        String versionName = checkNotNull(
-                (String) valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
-                KMSRESTConstants.VERSION_NAME_FIELD);
+  private static EncryptedKeyVersion parseJSONEncKeyVersion(String keyName,
+      Map valueMap) {
+    String versionName = checkNotNull(
+        (String) valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
+        KMSRESTConstants.VERSION_NAME_FIELD);
 
-        byte[] iv = Base64.decodeBase64(checkNotNull(
-                (String) valueMap.get(KMSRESTConstants.IV_FIELD),
-                KMSRESTConstants.IV_FIELD));
+    byte[] iv = Base64.decodeBase64(checkNotNull(
+        (String) valueMap.get(KMSRESTConstants.IV_FIELD),
+        KMSRESTConstants.IV_FIELD));
 
-        Map encValueMap = checkNotNull((Map)
-                valueMap.get(KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD),
-                KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD);
+    Map encValueMap = checkNotNull((Map)
+            valueMap.get(KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD),
+        KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD);
 
-        String encVersionName = checkNotNull((String)
-                encValueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
-                KMSRESTConstants.VERSION_NAME_FIELD);
+    String encVersionName = checkNotNull((String)
+            encValueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
+        KMSRESTConstants.VERSION_NAME_FIELD);
 
-        byte[] encKeyMaterial = Base64.decodeBase64(checkNotNull((String)
-                encValueMap.get(KMSRESTConstants.MATERIAL_FIELD),
-                KMSRESTConstants.MATERIAL_FIELD));
+    byte[] encKeyMaterial = Base64.decodeBase64(checkNotNull((String)
+            encValueMap.get(KMSRESTConstants.MATERIAL_FIELD),
+        KMSRESTConstants.MATERIAL_FIELD));
 
-        ekvs.add(new KMSEncryptedKeyVersion(keyName, versionName, iv,
-            encVersionName, encKeyMaterial));
-      }
-    }
-    return ekvs;
+    return new KMSEncryptedKeyVersion(keyName, versionName, iv,
+        encVersionName, encKeyMaterial);
   }
 
   private static KeyVersion parseJSONKeyVersion(Map valueMap) {
@@ -838,6 +842,35 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
   }
 
   @Override
+  public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+      throws IOException, GeneralSecurityException {
+    checkNotNull(ekv.getEncryptionKeyVersionName(), "versionName");
+    checkNotNull(ekv.getEncryptedKeyIv(), "iv");
+    checkNotNull(ekv.getEncryptedKeyVersion(), "encryptedKey");
+    Preconditions.checkArgument(ekv.getEncryptedKeyVersion().getVersionName()
+            .equals(KeyProviderCryptoExtension.EEK),
+        "encryptedKey version name must be '%s', is '%s'",
+        KeyProviderCryptoExtension.EEK,
+        ekv.getEncryptedKeyVersion().getVersionName());
+    final Map<String, String> params = new HashMap<>();
+    params.put(KMSRESTConstants.EEK_OP, KMSRESTConstants.EEK_REENCRYPT);
+    final Map<String, Object> jsonPayload = new HashMap<>();
+    jsonPayload.put(KMSRESTConstants.NAME_FIELD, ekv.getEncryptionKeyName());
+    jsonPayload.put(KMSRESTConstants.IV_FIELD,
+        Base64.encodeBase64String(ekv.getEncryptedKeyIv()));
+    jsonPayload.put(KMSRESTConstants.MATERIAL_FIELD,
+        Base64.encodeBase64String(ekv.getEncryptedKeyVersion().getMaterial()));
+    final URL url = createURL(KMSRESTConstants.KEY_VERSION_RESOURCE,
+        ekv.getEncryptionKeyVersionName(), KMSRESTConstants.EEK_SUB_RESOURCE,
+        params);
+    final HttpURLConnection conn = createConnection(url, HTTP_POST);
+    conn.setRequestProperty(CONTENT_TYPE, APPLICATION_JSON_MIME);
+    final Map response =
+        call(conn, jsonPayload, HttpURLConnection.HTTP_OK, Map.class);
+    return parseJSONEncKeyVersion(ekv.getEncryptionKeyName(), response);
+  }
+
+  @Override
   public List<KeyVersion> getKeyVersions(String name) throws IOException {
     checkNotEmpty(name, "name");
     URL url = createURL(KMSRESTConstants.KEY_RESOURCE, name,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
index b7d7898..3d7951e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
@@ -42,6 +42,7 @@ public class KMSRESTConstants {
   public static final String EEK_GENERATE = "generate";
   public static final String EEK_DECRYPT = "decrypt";
   public static final String EEK_NUM_KEYS = "num_keys";
+  public static final String EEK_REENCRYPT = "reencrypt";
 
   public static final String IV_FIELD = "iv";
   public static final String NAME_FIELD = "name";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
index e6ff079..86728d9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
@@ -218,6 +218,24 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     }
   }
 
+  public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion edek)
+      throws IOException, GeneralSecurityException {
+    try {
+      return doOp(new ProviderCallable<EncryptedKeyVersion>() {
+        @Override
+        public EncryptedKeyVersion call(KMSClientProvider provider)
+            throws IOException, GeneralSecurityException {
+          return provider.reencryptEncryptedKey(edek);
+        }
+      }, nextIdx());
+    } catch (WrapperException we) {
+      if (we.getCause() instanceof GeneralSecurityException) {
+        throw (GeneralSecurityException) we.getCause();
+      }
+      throw new IOException(we.getCause());
+    }
+  }
+
   @Override
   public KeyVersion getKeyVersion(final String versionName) throws IOException {
     return doOp(new ProviderCallable<KeyVersion>() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
index 4712698..32938e3 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
@@ -33,7 +33,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import static org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import static org.junit.Assert.assertArrayEquals;
@@ -52,6 +54,9 @@ public class TestKeyProviderCryptoExtension {
   private static KeyProvider.Options options;
   private static KeyVersion encryptionKey;
 
+  @Rule
+  public Timeout testTimeout = new Timeout(180000);
+
   @BeforeClass
   public static void setup() throws Exception {
     conf = new Configuration();
@@ -139,6 +144,103 @@ public class TestKeyProviderCryptoExtension {
   }
 
   @Test
+  public void testReencryptEncryptedKey() throws Exception {
+    // Generate a new EEK
+    final KeyProviderCryptoExtension.EncryptedKeyVersion ek1 =
+        kpExt.generateEncryptedKey(encryptionKey.getName());
+
+    // Decrypt EEK into an EK and check it
+    final KeyVersion k1 = kpExt.decryptEncryptedKey(ek1);
+    assertEquals(KeyProviderCryptoExtension.EK, k1.getVersionName());
+    assertEquals(encryptionKey.getMaterial().length, k1.getMaterial().length);
+    if (Arrays.equals(k1.getMaterial(), encryptionKey.getMaterial())) {
+      fail("Encrypted key material should not equal encryption key material");
+    }
+
+    // Roll the EK
+    kpExt.rollNewVersion(ek1.getEncryptionKeyName());
+
+    // Reencrypt ek1
+    final KeyProviderCryptoExtension.EncryptedKeyVersion ek2 =
+        kpExt.reencryptEncryptedKey(ek1);
+    assertEquals("Version name of EEK should be EEK",
+        KeyProviderCryptoExtension.EEK,
+        ek2.getEncryptedKeyVersion().getVersionName());
+    assertEquals("Name of EEK should be encryption key name",
+        ENCRYPTION_KEY_NAME, ek2.getEncryptionKeyName());
+    assertNotNull("Expected encrypted key material",
+        ek2.getEncryptedKeyVersion().getMaterial());
+    assertEquals("Length of encryption key material and EEK material should "
+            + "be the same", encryptionKey.getMaterial().length,
+        ek2.getEncryptedKeyVersion().getMaterial().length);
+    if (Arrays.equals(ek2.getEncryptedKeyVersion().getMaterial(),
+        encryptionKey.getMaterial())) {
+      fail("Encrypted key material should not equal decrypted key material");
+    }
+    if (Arrays.equals(ek2.getEncryptedKeyVersion().getMaterial(),
+        ek1.getEncryptedKeyVersion().getMaterial())) {
+      fail("Re-encrypted EEK should have different material");
+    }
+
+    // Decrypt the new EEK into an EK and check it
+    final KeyVersion k2 = kpExt.decryptEncryptedKey(ek2);
+    assertEquals(KeyProviderCryptoExtension.EK, k2.getVersionName());
+    assertEquals(encryptionKey.getMaterial().length, k2.getMaterial().length);
+    if (Arrays.equals(k2.getMaterial(), encryptionKey.getMaterial())) {
+      fail("Encrypted key material should not equal encryption key material");
+    }
+
+    // Re-encrypting the same EEK with the same EK should be deterministic
+    final KeyProviderCryptoExtension.EncryptedKeyVersion ek2a =
+        kpExt.reencryptEncryptedKey(ek1);
+    assertEquals("Version name of EEK should be EEK",
+        KeyProviderCryptoExtension.EEK,
+        ek2a.getEncryptedKeyVersion().getVersionName());
+    assertEquals("Name of EEK should be encryption key name",
+        ENCRYPTION_KEY_NAME, ek2a.getEncryptionKeyName());
+    assertNotNull("Expected encrypted key material",
+        ek2a.getEncryptedKeyVersion().getMaterial());
+    assertEquals("Length of encryption key material and EEK material should "
+            + "be the same", encryptionKey.getMaterial().length,
+        ek2a.getEncryptedKeyVersion().getMaterial().length);
+    if (Arrays.equals(ek2a.getEncryptedKeyVersion().getMaterial(),
+        encryptionKey.getMaterial())) {
+      fail("Encrypted key material should not equal decrypted key material");
+    }
+    if (Arrays.equals(ek2a.getEncryptedKeyVersion().getMaterial(),
+        ek1.getEncryptedKeyVersion().getMaterial())) {
+      fail("Re-encrypted EEK should have different material");
+    }
+    assertArrayEquals(ek2.getEncryptedKeyVersion().getMaterial(),
+        ek2a.getEncryptedKeyVersion().getMaterial());
+
+    // Re-encrypting an EEK with the same version EK should be no-op
+    final KeyProviderCryptoExtension.EncryptedKeyVersion ek3 =
+        kpExt.reencryptEncryptedKey(ek2);
+    assertEquals("Version name of EEK should be EEK",
+        KeyProviderCryptoExtension.EEK,
+        ek3.getEncryptedKeyVersion().getVersionName());
+    assertEquals("Name of EEK should be encryption key name",
+        ENCRYPTION_KEY_NAME, ek3.getEncryptionKeyName());
+    assertNotNull("Expected encrypted key material",
+        ek3.getEncryptedKeyVersion().getMaterial());
+    assertEquals("Length of encryption key material and EEK material should "
+            + "be the same", encryptionKey.getMaterial().length,
+        ek3.getEncryptedKeyVersion().getMaterial().length);
+    if (Arrays.equals(ek3.getEncryptedKeyVersion().getMaterial(),
+        encryptionKey.getMaterial())) {
+      fail("Encrypted key material should not equal decrypted key material");
+    }
+
+    if (Arrays.equals(ek3.getEncryptedKeyVersion().getMaterial(),
+        ek1.getEncryptedKeyVersion().getMaterial())) {
+      fail("Re-encrypted EEK should have different material");
+    }
+    assertArrayEquals(ek2.getEncryptedKeyVersion().getMaterial(),
+        ek3.getEncryptedKeyVersion().getMaterial());
+  }
+
+  @Test
   public void testNonDefaultCryptoExtensionSelectionWithCachingKeyProvider()
           throws Exception {
     Configuration config = new Configuration();
@@ -234,6 +336,12 @@ public class TestKeyProviderCryptoExtension {
     }
 
     @Override
+    public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+        throws IOException, GeneralSecurityException {
+      return ekv;
+    }
+
+    @Override
     public KeyVersion decryptEncryptedKey(
             EncryptedKeyVersion encryptedKeyVersion)
             throws IOException, GeneralSecurityException {
@@ -339,5 +447,11 @@ public class TestKeyProviderCryptoExtension {
             throws IOException, GeneralSecurityException {
       return kv;
     }
+
+    @Override
+    public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+        throws IOException, GeneralSecurityException {
+      return ekv;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/EagerKeyGeneratorKeyProviderCryptoExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/EagerKeyGeneratorKeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/EagerKeyGeneratorKeyProviderCryptoExtension.java
index 04cd065..f79872d 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/EagerKeyGeneratorKeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/EagerKeyGeneratorKeyProviderCryptoExtension.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ExecutionException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.kms.ValueQueue;
 import org.apache.hadoop.crypto.key.kms.ValueQueue.SyncGenerationPolicy;
@@ -136,6 +135,12 @@ public class EagerKeyGeneratorKeyProviderCryptoExtension
       return keyProviderCryptoExtension.decryptEncryptedKey(
           encryptedKeyVersion);
     }
+
+    @Override
+    public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+        throws IOException, GeneralSecurityException {
+      return keyProviderCryptoExtension.reencryptEncryptedKey(ekv);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
index 271c339..9b8b977 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
@@ -65,7 +64,7 @@ public class KMS {
     CREATE_KEY, DELETE_KEY, ROLL_NEW_VERSION,
     GET_KEYS, GET_KEYS_METADATA,
     GET_KEY_VERSIONS, GET_METADATA, GET_KEY_VERSION, GET_CURRENT_KEY,
-    GENERATE_EEK, DECRYPT_EEK
+    GENERATE_EEK, DECRYPT_EEK, REENCRYPT_EEK
   }
 
   private KeyProviderCryptoExtension provider;
@@ -510,7 +509,7 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_VERSION_RESOURCE + "/{versionName:.*}/" +
       KMSRESTConstants.EEK_SUB_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
-  public Response decryptEncryptedKey(
+  public Response handleEncryptedKeyOp(
       @PathParam("versionName") final String versionName,
       @QueryParam(KMSRESTConstants.EEK_OP) String eekOp,
       Map jsonPayload)
@@ -528,15 +527,15 @@ public class KMS {
       String ivStr = (String) jsonPayload.get(KMSRESTConstants.IV_FIELD);
       String encMaterialStr =
               (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
+      KMSClientProvider.checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
+      final byte[] iv = Base64.decodeBase64(ivStr);
+      KMSClientProvider.checkNotNull(encMaterialStr,
+          KMSRESTConstants.MATERIAL_FIELD);
+      final byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
       Object retJSON;
       if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
         assertAccess(KMSACLs.Type.DECRYPT_EEK, user, KMSOp.DECRYPT_EEK,
                 keyName);
-        KMSClientProvider.checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
-        final byte[] iv = Base64.decodeBase64(ivStr);
-        KMSClientProvider.checkNotNull(encMaterialStr,
-                KMSRESTConstants.MATERIAL_FIELD);
-        final byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
 
         KeyProvider.KeyVersion retKeyVersion = user.doAs(
                 new PrivilegedExceptionAction<KeyVersion>() {
@@ -554,6 +553,23 @@ public class KMS {
 
         retJSON = KMSServerJSONUtils.toJSON(retKeyVersion);
         kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
+      } else if (eekOp.equals(KMSRESTConstants.EEK_REENCRYPT)) {
+        assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.REENCRYPT_EEK,
+            keyName);
+
+        EncryptedKeyVersion retEncryptedKeyVersion =
+            user.doAs(new PrivilegedExceptionAction<EncryptedKeyVersion>() {
+              @Override
+              public EncryptedKeyVersion run() throws Exception {
+                return provider.reencryptEncryptedKey(
+                    new KMSClientProvider.KMSEncryptedKeyVersion(keyName,
+                        versionName, iv, KeyProviderCryptoExtension.EEK,
+                        encMaterial));
+              }
+            });
+
+        retJSON = KMSServerJSONUtils.toJSON(retEncryptedKeyVersion);
+        kmsAudit.ok(user, KMSOp.REENCRYPT_EEK, keyName, "");
       } else {
         StringBuilder error;
         error = new StringBuilder("IllegalArgumentException Wrong ");
@@ -566,11 +582,11 @@ public class KMS {
         throw new IllegalArgumentException(error.toString());
       }
       KMSWebApp.getDecryptEEKCallsMeter().mark();
-      LOG.trace("Exiting decryptEncryptedKey method.");
+      LOG.trace("Exiting handleEncryptedKeyOp method.");
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
-              .build();
+          .build();
     } catch (Exception e) {
-      LOG.debug("Exception in decryptEncryptedKey.", e);
+      LOG.debug("Exception in handleEncryptedKeyOp.", e);
       throw e;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
index eef0bce..7fdfc34 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
@@ -55,8 +55,8 @@ import java.util.Set;
 public class KMSAudit {
   @VisibleForTesting
   static final Set<KMS.KMSOp> AGGREGATE_OPS_WHITELIST = Sets.newHashSet(
-    KMS.KMSOp.GET_KEY_VERSION, KMS.KMSOp.GET_CURRENT_KEY,
-    KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK
+      KMS.KMSOp.GET_KEY_VERSION, KMS.KMSOp.GET_CURRENT_KEY,
+      KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK, KMS.KMSOp.REENCRYPT_EEK
   );
 
   private Cache<String, AuditEvent> cache;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
index e8cabc8..b0e85da 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
@@ -265,6 +265,19 @@ public class KeyAuthorizationKeyProvider extends KeyProviderCryptoExtension {
   }
 
   @Override
+  public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+      throws IOException, GeneralSecurityException {
+    readLock.lock();
+    try {
+      verifyKeyVersionBelongsToKey(ekv);
+      doAccessCheck(ekv.getEncryptionKeyName(), KeyOpType.GENERATE_EEK);
+      return provider.reencryptEncryptedKey(ekv);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
   public KeyVersion getKeyVersion(String versionName) throws IOException {
     readLock.lock();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
index 8fd0f58..58c1fbd 100644
--- a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
+++ b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
@@ -169,7 +169,7 @@ Client-side can be changed via the following properties in the `etc/hadoop/core-
 
 $H3 KMS Aggregated Audit logs
 
-Audit logs are aggregated for API accesses to the GET\_KEY\_VERSION, GET\_CURRENT\_KEY, DECRYPT\_EEK, GENERATE\_EEK operations.
+Audit logs are aggregated for API accesses to the GET\_KEY\_VERSION, GET\_CURRENT\_KEY, DECRYPT\_EEK, GENERATE\_EEK, REENCRYPT\_EEK operations.
 
 Entries are grouped by the (user,key,operation) combined key for a configurable aggregation interval after which the number of accesses to the specified end-point by the user for a given key is flushed to the audit log.
 
@@ -481,7 +481,7 @@ $H5 Key ACLs
 KMS supports access control for all non-read operations at the Key level. All Key Access operations are classified as :
 
 * MANAGEMENT - createKey, deleteKey, rolloverNewVersion
-* GENERATE_EEK - generateEncryptedKey, warmUpEncryptedKeys
+* GENERATE_EEK - generateEncryptedKey, reencryptEncryptedKey, warmUpEncryptedKeys
 * DECRYPT_EEK - decryptEncryptedKey
 * READ - getKeyVersion, getKeyVersions, getMetadata, getKeysMetadata, getCurrentKey
 * ALL - all of the above
@@ -876,7 +876,7 @@ $H4 Generate Encrypted Key for Current KeyVersion
     Content-Type: application/json
     [
       {
-        "versionName"         : "encryptionVersionName",
+        "versionName"         : "<encryptionVersionName>",
         "iv"                  : "<iv>",          //base64
         "encryptedKeyVersion" : {
             "versionName"       : "EEK",
@@ -884,7 +884,7 @@ $H4 Generate Encrypted Key for Current KeyVersion
         }
       },
       {
-        "versionName"         : "encryptionVersionName",
+        "versionName"         : "<encryptionVersionName>",
         "iv"                  : "<iv>",          //base64
         "encryptedKeyVersion" : {
             "versionName"       : "EEK",
@@ -917,6 +917,37 @@ $H4 Decrypt Encrypted Key
       "material"    : "<material>",    //base64
     }
 
+$H4 Re-encrypt Encrypted Key With The Latest KeyVersion
+
+This command takes a previously generated encrypted key, and re-encrypts it using the latest KeyVersion encryption key in the KeyProvider. If the latest KeyVersion is the same as the one used to generate the encrypted key, the same encrypted key is returned.
+
+This is usually useful after a [Rollover](Rollover_Key) of an encryption key. Re-encrypting the encrypted key will allow it to be encrypted using the latest version of the encryption key, but still with the same key material and initialization vector.
+
+*REQUEST:*
+
+    POST http://HOST:PORT/kms/v1/keyversion/<version-name>/_eek?eek_op=reencrypt
+    Content-Type: application/json
+
+    {
+      "name"        : "<key-name>",
+      "iv"          : "<iv>",          //base64
+      "material"    : "<material>",    //base64
+    }
+
+*RESPONSE:*
+
+    200 OK
+    Content-Type: application/json
+
+    {
+      "versionName"         : "<encryptionVersionName>",
+      "iv"                  : "<iv>",            //base64
+      "encryptedKeyVersion" : {
+          "versionName"       : "EEK",
+          "material"          : "<material>",    //base64
+      }
+    }
+
 $H4 Get Key Version
 
 *REQUEST:*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index aaab172..3a2d53c 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -68,6 +68,7 @@ import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
@@ -79,6 +80,11 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+
 public class TestKMS {
   private static final Logger LOG = LoggerFactory.getLogger(TestKMS.class);
 
@@ -462,6 +468,7 @@ public class TestKMS {
   }
 
   @Test
+  @SuppressWarnings("checkstyle:methodlength")
   public void testKMSProvider() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
@@ -606,6 +613,25 @@ public class TestKMS {
         }
         Assert.assertFalse(isEq);
 
+        // test re-encrypt
+        kpExt.rollNewVersion(ek1.getEncryptionKeyName());
+        EncryptedKeyVersion ek1r = kpExt.reencryptEncryptedKey(ek1);
+        assertEquals(KeyProviderCryptoExtension.EEK,
+            ek1r.getEncryptedKeyVersion().getVersionName());
+        assertFalse(Arrays.equals(ek1.getEncryptedKeyVersion().getMaterial(),
+            ek1r.getEncryptedKeyVersion().getMaterial()));
+        assertEquals(kv.getMaterial().length,
+            ek1r.getEncryptedKeyVersion().getMaterial().length);
+        assertEquals(ek1.getEncryptionKeyName(), ek1r.getEncryptionKeyName());
+        assertArrayEquals(ek1.getEncryptedKeyIv(), ek1r.getEncryptedKeyIv());
+        assertNotEquals(ek1.getEncryptionKeyVersionName(),
+            ek1r.getEncryptionKeyVersionName());
+
+        KeyProvider.KeyVersion k1r = kpExt.decryptEncryptedKey(ek1r);
+        assertEquals(KeyProviderCryptoExtension.EK, k1r.getVersionName());
+        assertArrayEquals(k1.getMaterial(), k1r.getMaterial());
+        assertEquals(kv.getMaterial().length, k1r.getMaterial().length);
+
         // deleteKey()
         kp.deleteKey("k1");
 
@@ -721,6 +747,7 @@ public class TestKMS {
   }
 
   @Test
+  @SuppressWarnings("checkstyle:methodlength")
   public void testKeyACLs() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
@@ -969,17 +996,10 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             KeyProvider kp = createProvider(uri, conf);
-            try {
-              KeyProviderCryptoExtension kpce =
-                  KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
-              try {
-                kpce.generateEncryptedKey("k1");
-              } catch (Exception e) {
-                Assert.fail("User [GENERATE_EEK] should be allowed to generate_eek on k1");
-              }
-            } catch (Exception ex) {
-              Assert.fail(ex.getMessage());
-            }
+            KeyProviderCryptoExtension kpce =
+                KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
+            EncryptedKeyVersion ekv = kpce.generateEncryptedKey("k1");
+            kpce.reencryptEncryptedKey(ekv);
             return null;
           }
         });
@@ -1170,6 +1190,7 @@ public class TestKMS {
   }
 
   @Test
+  @SuppressWarnings("checkstyle:methodlength")
   public void testACLs() throws Exception {
     Configuration conf = new Configuration();
     conf.set("hadoop.security.authentication", "kerberos");
@@ -1401,6 +1422,17 @@ public class TestKMS {
           }
         });
 
+        doAs("GENERATE_EEK", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            KeyProvider kp = createProvider(uri, conf);
+            KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
+                createKeyProviderCryptoExtension(kp);
+            kpCE.reencryptEncryptedKey(encKv);
+            return null;
+          }
+        });
+
         doAs("DECRYPT_EEK", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
@@ -1449,6 +1481,7 @@ public class TestKMS {
         // test ACL reloading
         Thread.sleep(10); // to ensure the ACLs file modifiedTime is newer
         conf.set(KMSACLs.Type.CREATE.getAclConfigKey(), "foo");
+        conf.set(KMSACLs.Type.GENERATE_EEK.getAclConfigKey(), "foo");
         writeConf(testDir, conf);
         Thread.sleep(1000);
 
@@ -1473,6 +1506,41 @@ public class TestKMS {
           }
         });
 
+        doAs("GENERATE_EEK", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            KeyProvider kp = createProvider(uri, conf);
+            try {
+              KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
+                  createKeyProviderCryptoExtension(kp);
+              kpCE.generateEncryptedKey("k1");
+            } catch (IOException ex) {
+              // This isn't an AuthorizationException because generate goes
+              // through the ValueQueue. See KMSCP#generateEncryptedKey.
+              if (ex.getCause().getCause() instanceof AuthorizationException) {
+                LOG.info("Caught expected exception.", ex);
+              } else {
+                throw ex;
+              }
+            }
+            return null;
+          }
+        });
+
+        doAs("GENERATE_EEK", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            KeyProvider kp = createProvider(uri, conf);
+            try {
+              KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
+                  createKeyProviderCryptoExtension(kp);
+              kpCE.reencryptEncryptedKey(encKv);
+            } catch (AuthorizationException ex) {
+              LOG.info("Caught expected exception.", ex);
+            }
+            return null;
+          }
+        });
         return null;
       }
     });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79d90b81/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
index 2d9443f..ed4825c 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
@@ -95,6 +95,7 @@ public class TestKMSAudit {
   }
 
   @Test
+  @SuppressWarnings("checkstyle:linelength")
   public void testAggregation() throws Exception {
     UserGroupInformation luser = Mockito.mock(UserGroupInformation.class);
     Mockito.when(luser.getShortUserName()).thenReturn("luser");
@@ -109,6 +110,10 @@ public class TestKMSAudit {
     kmsAudit.evictCacheForTesting();
     kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
     kmsAudit.evictCacheForTesting();
+    kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK, "k1", "testmsg");
+    kmsAudit.evictCacheForTesting();
     String out = getAndResetLogOutput();
     System.out.println(out);
     Assert.assertTrue(
@@ -119,10 +124,13 @@ public class TestKMSAudit {
             + "OK\\[op=ROLL_NEW_VERSION, key=k1, user=luser\\] testmsg"
             // Aggregated
             + "OK\\[op=DECRYPT_EEK, key=k1, user=luser, accessCount=6, interval=[^m]{1,4}ms\\] testmsg"
-            + "OK\\[op=DECRYPT_EEK, key=k1, user=luser, accessCount=1, interval=[^m]{1,4}ms\\] testmsg"));
+            + "OK\\[op=DECRYPT_EEK, key=k1, user=luser, accessCount=1, interval=[^m]{1,4}ms\\] testmsg"
+            + "OK\\[op=REENCRYPT_EEK, key=k1, user=luser, accessCount=1, interval=[^m]{1,4}ms\\] testmsg"
+            + "OK\\[op=REENCRYPT_EEK, key=k1, user=luser, accessCount=3, interval=[^m]{1,4}ms\\] testmsg"));
   }
 
   @Test
+  @SuppressWarnings("checkstyle:linelength")
   public void testAggregationUnauth() throws Exception {
     UserGroupInformation luser = Mockito.mock(UserGroupInformation.class);
     Mockito.when(luser.getShortUserName()).thenReturn("luser");
@@ -159,6 +167,7 @@ public class TestKMSAudit {
   }
 
   @Test
+  @SuppressWarnings("checkstyle:linelength")
   public void testAuditLogFormat() throws Exception {
     UserGroupInformation luser = Mockito.mock(UserGroupInformation.class);
     Mockito.when(luser.getShortUserName()).thenReturn("luser");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/50] [abbrv] hadoop git commit: HADOOP-13835. Move Google Test Framework code from mapreduce to hadoop-common. Contributed by Varun Vasudev.

Posted by st...@apache.org.
HADOOP-13835. Move Google Test Framework code from mapreduce to hadoop-common. Contributed by Varun Vasudev.


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

Branch: refs/heads/HADOOP-13345
Commit: b2a3d6c519d83283a49b0d2172dcf1de97f9c4bc
Parents: 8e63fa9
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Dec 6 14:01:47 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Dec 6 14:01:47 2016 +0900

----------------------------------------------------------------------
 LICENSE.txt                                     |     2 +-
 hadoop-common-project/hadoop-common/pom.xml     |     1 +
 .../src/main/native/gtest/gtest-all.cc          | 10403 ++++++++
 .../src/main/native/gtest/include/gtest/gtest.h | 21192 +++++++++++++++++
 .../hadoop-mapreduce-client-nativetask/pom.xml  |     2 -
 .../src/CMakeLists.txt                          |     8 +-
 .../src/main/native/gtest/gtest-all.cc          | 10403 --------
 .../src/main/native/gtest/include/gtest/gtest.h | 21192 -----------------
 8 files changed, 31603 insertions(+), 31600 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 04d2daa..2183f0e 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -289,7 +289,7 @@ For src/main/native/src/org/apache/hadoop/io/compress/lz4/{lz4.h,lz4.c,lz4hc.h,l
 */
 
 
-For hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest
+For hadoop-common-project/hadoop-common/src/main/native/gtest
 ---------------------------------------------------------------------
 Copyright 2008, Google Inc.
 All rights reserved.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2a3d6c5/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 1d45452..c9b282f 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -523,6 +523,7 @@
             <exclude>src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc.h</exclude>
             <exclude>src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc.c</exclude>
             <exclude>src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc_encoder.h</exclude>
+            <exclude>src/main/native/gtest/**/*</exclude>
             <exclude>src/test/resources/test-untar.tgz</exclude>
             <exclude>src/test/resources/test.har/_SUCCESS</exclude>
             <exclude>src/test/resources/test.har/_index</exclude>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/50] [abbrv] hadoop git commit: MAPREDUCE-6816. Progress bars in Web UI always at 100%. Contributed by Shen Yinjie.

Posted by st...@apache.org.
MAPREDUCE-6816. Progress bars in Web UI always at 100%. Contributed by Shen Yinjie.


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

Branch: refs/heads/HADOOP-13345
Commit: da4ecc9f8223ac82bff8483594fb29be2f8d4d34
Parents: 6b4a997
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Dec 7 06:41:08 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Dec 7 06:41:08 2016 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java | 2 +-
 .../org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java    | 2 +-
 .../org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java   | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ecc9f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
index f8637e7..01c5b0d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
@@ -125,7 +125,7 @@ public class TaskPage extends AppView {
 
       for (TaskAttempt attempt : getTaskAttempts()) {
         TaskAttemptInfo ta = new TaskAttemptInfo(attempt, true);
-        String progress = StringUtils.formatPercent(ta.getProgress() / 100, 2);
+        String progress = StringUtils.format("%.2f", ta.getProgress());
 
         String nodeHttpAddr = ta.getNode();
         String diag = ta.getNote() == null ? "" : ta.getNote();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ecc9f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
index 7c1aa49..78338ec 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
@@ -94,7 +94,7 @@ public class TasksBlock extends HtmlBlock {
 
       TaskInfo info = new TaskInfo(task);
       String tid = info.getId();
-      String pct = StringUtils.formatPercent(info.getProgress() / 100, 2);
+      String pct = StringUtils.format("%.2f", info.getProgress());
       tasksTableData.append("[\"<a href='").append(url("task", tid))
       .append("'>").append(tid).append("</a>\",\"")
       //Progress bar

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da4ecc9f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
index ffb243b..3ed65b9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
@@ -104,12 +104,12 @@ public class JobInfo {
     this.mapsCompleted = job.getCompletedMaps();
     this.mapProgress = report.getMapProgress() * 100;
     this.mapProgressPercent =
-        StringUtils.formatPercent(report.getMapProgress(), 2);
+        StringUtils.format("%.2f", getMapProgress());
     this.reducesTotal = job.getTotalReduces();
     this.reducesCompleted = job.getCompletedReduces();
     this.reduceProgress = report.getReduceProgress() * 100;
     this.reduceProgressPercent =
-        StringUtils.formatPercent(report.getReduceProgress(), 2);
+        StringUtils.format("%.2f", getReduceProgress());
 
     this.acls = new ArrayList<ConfEntryInfo>();
     if (hasAccess) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/50] [abbrv] hadoop git commit: MAPREDUCE-6571. JobEndNotification info logs are missing in AM container syslog (haibochen via rkanter)

Posted by st...@apache.org.
MAPREDUCE-6571. JobEndNotification info logs are missing in AM container syslog (haibochen via rkanter)


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

Branch: refs/heads/HADOOP-13345
Commit: 6b4a9970f187ab399adf7469e494a530f08993b5
Parents: f53b481
Author: Robert Kanter <rk...@apache.org>
Authored: Tue Dec 6 13:11:03 2016 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Tue Dec 6 13:11:03 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/mapreduce/v2/app/MRAppMaster.java   | 11 -----------
 .../org/apache/hadoop/mapreduce/v2/app/MRApp.java     | 14 --------------
 2 files changed, 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b4a9970/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index b383a02..78c8bdd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -151,7 +151,6 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.log4j.LogManager;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -1277,14 +1276,9 @@ public class MRAppMaster extends CompositeService {
     }
   }
 
-  protected void shutdownTaskLog() {
-    TaskLog.syncLogsShutdown(logSyncer);
-  }
-
   @Override
   public void stop() {
     super.stop();
-    shutdownTaskLog();
   }
 
   private boolean isRecoverySupported() throws IOException {
@@ -1824,14 +1818,9 @@ public class MRAppMaster extends CompositeService {
     T call(Configuration conf) throws Exception;
   }
 
-  protected void shutdownLogManager() {
-    LogManager.shutdown();
-  }
-
   @Override
   protected void serviceStop() throws Exception {
     super.serviceStop();
-    shutdownLogManager();
   }
 
   public ClientService getClientService() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b4a9970/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
index 3472b1f..2d0b5ea 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
@@ -816,19 +816,5 @@ public class MRApp extends MRAppMaster {
     return token.decodeIdentifier();
   }
 
-  @Override
-  protected void shutdownTaskLog() {
-    // Avoid closing the logging system during unit tests,
-    // otherwise subsequent MRApp instances in the same test
-    // will fail to log anything.
-  }
-
-  @Override
-  protected void shutdownLogManager() {
-    // Avoid closing the logging system during unit tests,
-    // otherwise subsequent MRApp instances in the same test
-    // will fail to log anything.
-  }
-
 }
  


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/50] [abbrv] hadoop git commit: YARN-5965. Retrospect ApplicationReport#getApplicationTimeouts. Contributed by Rohith Sharma K S

Posted by st...@apache.org.
YARN-5965. Retrospect ApplicationReport#getApplicationTimeouts. Contributed by Rohith Sharma K S


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

Branch: refs/heads/HADOOP-13345
Commit: ab923a53fcf55d4d75aa027d46e3c4a659015325
Parents: c73e08a
Author: Sunil <su...@apache.org>
Authored: Thu Dec 8 00:27:25 2016 +0530
Committer: Sunil <su...@apache.org>
Committed: Thu Dec 8 00:27:25 2016 +0530

----------------------------------------------------------------------
 .../yarn/api/records/ApplicationReport.java     |   6 +-
 .../src/main/proto/yarn_protos.proto            |   7 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   3 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   7 +-
 .../impl/pb/ApplicationReportPBImpl.java        | 107 +++++++++++--------
 .../server/resourcemanager/rmapp/RMAppImpl.java |  13 ++-
 .../rmapp/TestApplicationLifetimeMonitor.java   |  15 ++-
 7 files changed, 91 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab923a53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
index c039514..9e9ec3c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
-import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -451,10 +451,10 @@ public abstract class ApplicationReport {
 
   @Public
   @Unstable
-  public abstract List<ApplicationTimeout> getApplicationTimeouts();
+  public abstract Map<ApplicationTimeoutType, ApplicationTimeout> getApplicationTimeouts();
 
   @Private
   @Unstable
   public abstract void setApplicationTimeouts(
-      List<ApplicationTimeout> timeouts);
+      Map<ApplicationTimeoutType, ApplicationTimeout> timeouts);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab923a53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 43a661f..5a70298 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -214,7 +214,12 @@ message ApplicationReportProto {
   optional PriorityProto priority = 23;
   optional string appNodeLabelExpression = 24;
   optional string amNodeLabelExpression = 25;
-  repeated ApplicationTimeoutProto application_timeouts = 26;
+  repeated AppTimeoutsMapProto appTimeouts = 26;
+}
+
+message AppTimeoutsMapProto {
+  optional ApplicationTimeoutTypeProto application_timeout_type = 1;
+  optional ApplicationTimeoutProto application_timeout = 2;
 }
 
 message ApplicationTimeoutProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab923a53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index 20a65bf..efe5921 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -714,7 +714,8 @@ public class ApplicationCLI extends YarnCLI {
       appReportStr.println(appReport.getAppNodeLabelExpression());
       appReportStr.print("\tAM container Node Label Expression : ");
       appReportStr.println(appReport.getAmNodeLabelExpression());
-      for (ApplicationTimeout timeout : appReport.getApplicationTimeouts()) {
+      for (ApplicationTimeout timeout : appReport.getApplicationTimeouts()
+          .values()) {
         appReportStr.print("\tTimeoutType : " + timeout.getTimeoutType());
         appReportStr.print("\tExpiryTime : " + timeout.getExpiryTime());
         appReportStr.println(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab923a53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index f9ec5c7..a677606 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -130,8 +130,8 @@ public class TestYarnCLI {
       newApplicationReport.setPriority(Priority.newInstance(0));
       ApplicationTimeout timeout = ApplicationTimeout
           .newInstance(ApplicationTimeoutType.LIFETIME, "UNLIMITED", -1);
-      newApplicationReport
-          .setApplicationTimeouts(Collections.singletonList(timeout));
+      newApplicationReport.setApplicationTimeouts(
+          Collections.singletonMap(timeout.getTimeoutType(), timeout));
 
       when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
           newApplicationReport);
@@ -2104,7 +2104,8 @@ public class TestYarnCLI {
         "N/A", 0.53789f, "YARN", null);
     ApplicationTimeout timeout = ApplicationTimeout
         .newInstance(ApplicationTimeoutType.LIFETIME, "N/A", -1);
-    appReport.setApplicationTimeouts(Collections.singletonList(timeout));
+    appReport.setApplicationTimeouts(
+        Collections.singletonMap(timeout.getTimeoutType(), timeout));
     when(client.getApplicationReport(any(ApplicationId.class)))
         .thenReturn(appReport);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab923a53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
index f4987d3..036fa90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
@@ -26,11 +26,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeout;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.proto.YarnProtos.AppTimeoutsMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationReportProto;
@@ -44,10 +46,11 @@ import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 
 import com.google.protobuf.TextFormat;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 @Private
@@ -63,7 +66,7 @@ public class ApplicationReportPBImpl extends ApplicationReport {
   private Token amRmToken = null;
   private Set<String> applicationTags = null;
   private Priority priority = null;
-  private List<ApplicationTimeout> applicationTimeoutList = null;
+  private Map<ApplicationTimeoutType, ApplicationTimeout> applicationTimeouts = null;
 
   public ApplicationReportPBImpl() {
     builder = ApplicationReportProto.newBuilder();
@@ -498,8 +501,8 @@ public class ApplicationReportPBImpl extends ApplicationReport {
             builder.getPriority())) {
       builder.setPriority(convertToProtoFormat(this.priority));
     }
-    if (this.applicationTimeoutList != null) {
-      addLocalApplicationTimeoutToProto();
+    if (this.applicationTimeouts != null) {
+      addApplicationTimeouts();
     }
   }
 
@@ -679,76 +682,86 @@ public class ApplicationReportPBImpl extends ApplicationReport {
   }
 
   @Override
-  public List<ApplicationTimeout> getApplicationTimeouts() {
-    initLocalApplicationsList();
-    return this.applicationTimeoutList;
+  public Map<ApplicationTimeoutType, ApplicationTimeout> getApplicationTimeouts() {
+    initApplicationTimeout();
+    return this.applicationTimeouts;
   }
 
-  private void initLocalApplicationsList() {
-    if (this.applicationTimeoutList != null) {
+  @Override
+  public void setApplicationTimeouts(
+      Map<ApplicationTimeoutType, ApplicationTimeout> timeouts) {
+    if (timeouts == null) {
       return;
     }
-    ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
-    List<ApplicationTimeoutProto> list = p.getApplicationTimeoutsList();
-    this.applicationTimeoutList = new ArrayList<ApplicationTimeout>();
+    initApplicationTimeout();
+    this.applicationTimeouts.clear();
+    this.applicationTimeouts.putAll(timeouts);
+  }
 
-    for (ApplicationTimeoutProto a : list) {
-      this.applicationTimeoutList.add(convertFromProtoFormat(a));
+  private void initApplicationTimeout() {
+    if (this.applicationTimeouts != null) {
+      return;
+    }
+    ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
+    List<AppTimeoutsMapProto> lists = p.getAppTimeoutsList();
+    this.applicationTimeouts =
+        new HashMap<ApplicationTimeoutType, ApplicationTimeout>(lists.size());
+    for (AppTimeoutsMapProto timeoutProto : lists) {
+      this.applicationTimeouts.put(
+          ProtoUtils
+              .convertFromProtoFormat(timeoutProto.getApplicationTimeoutType()),
+          convertFromProtoFormat(timeoutProto.getApplicationTimeout()));
     }
   }
 
-  private void addLocalApplicationTimeoutToProto() {
+  private ApplicationTimeoutPBImpl convertFromProtoFormat(
+      ApplicationTimeoutProto p) {
+    return new ApplicationTimeoutPBImpl(p);
+  }
+
+  private ApplicationTimeoutProto convertToProtoFormat(ApplicationTimeout t) {
+    return ((ApplicationTimeoutPBImpl) t).getProto();
+  }
+
+  private void addApplicationTimeouts() {
     maybeInitBuilder();
-    builder.clearApplicationTimeouts();
-    if (applicationTimeoutList == null) {
+    builder.clearAppTimeouts();
+    if (applicationTimeouts == null) {
       return;
     }
-    Iterable<ApplicationTimeoutProto> iterable =
-        new Iterable<ApplicationTimeoutProto>() {
-          @Override
-          public Iterator<ApplicationTimeoutProto> iterator() {
-            return new Iterator<ApplicationTimeoutProto>() {
+    Iterable<? extends AppTimeoutsMapProto> values =
+        new Iterable<AppTimeoutsMapProto>() {
 
-              private Iterator<ApplicationTimeout> iter =
-                  applicationTimeoutList.iterator();
+          @Override
+          public Iterator<AppTimeoutsMapProto> iterator() {
+            return new Iterator<AppTimeoutsMapProto>() {
+              private Iterator<ApplicationTimeoutType> iterator =
+                  applicationTimeouts.keySet().iterator();
 
               @Override
               public boolean hasNext() {
-                return iter.hasNext();
+                return iterator.hasNext();
               }
 
               @Override
-              public ApplicationTimeoutProto next() {
-                return convertToProtoFormat(iter.next());
+              public AppTimeoutsMapProto next() {
+                ApplicationTimeoutType key = iterator.next();
+                return AppTimeoutsMapProto.newBuilder()
+                    .setApplicationTimeout(
+                        convertToProtoFormat(applicationTimeouts.get(key)))
+                    .setApplicationTimeoutType(
+                        ProtoUtils.convertToProtoFormat(key))
+                    .build();
               }
 
               @Override
               public void remove() {
                 throw new UnsupportedOperationException();
-
               }
             };
-
           }
         };
-    builder.addAllApplicationTimeouts(iterable);
+    this.builder.addAllAppTimeouts(values);
   }
 
-  private ApplicationTimeoutPBImpl convertFromProtoFormat(
-      ApplicationTimeoutProto p) {
-    return new ApplicationTimeoutPBImpl(p);
-  }
-
-  private ApplicationTimeoutProto convertToProtoFormat(ApplicationTimeout t) {
-    return ((ApplicationTimeoutPBImpl) t).getProto();
-  }
-
-  @Override
-  public void setApplicationTimeouts(List<ApplicationTimeout> timeouts) {
-    maybeInitBuilder();
-    if (timeouts == null) {
-      builder.clearApplicationTimeouts();
-    }
-    this.applicationTimeoutList = timeouts;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab923a53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 1f1586a..a647969 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -790,10 +790,17 @@ public class RMAppImpl implements RMApp, Recoverable {
         long timeoutInMillis = applicationTimeouts
             .get(ApplicationTimeoutType.LIFETIME).longValue();
         timeout.setExpiryTime(Times.formatISO8601(timeoutInMillis));
-        timeout.setRemainingTime(
-            Math.max((timeoutInMillis - systemClock.getTime()) / 1000, 0));
+        if (isAppInCompletedStates()) {
+          // if application configured with timeout and finished before timeout
+          // happens then remaining time should not be calculated.
+          timeout.setRemainingTime(0);
+        } else {
+          timeout.setRemainingTime(
+              Math.max((timeoutInMillis - systemClock.getTime()) / 1000, 0));
+        }
       }
-      report.setApplicationTimeouts(Collections.singletonList(timeout));
+      report.setApplicationTimeouts(
+          Collections.singletonMap(timeout.getTimeoutType(), timeout));
       return report;
     } finally {
       this.readLock.unlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab923a53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
index aa4acca..fdc47b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -136,15 +135,13 @@ public class TestApplicationLifetimeMonitor {
       GetApplicationReportRequest appRequest =
           recordFactory.newRecordInstance(GetApplicationReportRequest.class);
       appRequest.setApplicationId(app2.getApplicationId());
-      List<ApplicationTimeout> applicationTimeoutList = rm.getRMContext()
-          .getClientRMService().getApplicationReport(appRequest)
+      Map<ApplicationTimeoutType, ApplicationTimeout> appTimeouts = rm
+          .getRMContext().getClientRMService().getApplicationReport(appRequest)
           .getApplicationReport().getApplicationTimeouts();
-      Assert.assertTrue("Application Timeout list are empty.",
-          !applicationTimeoutList.isEmpty());
-      ApplicationTimeout timeout = applicationTimeoutList.iterator().next();
-      Assert.assertEquals("Application timeout Type is incorrect.",
-          ApplicationTimeoutType.LIFETIME.toString(),
-          timeout.getTimeoutType().toString());
+      Assert.assertTrue("Application Timeout are empty.",
+          !appTimeouts.isEmpty());
+      ApplicationTimeout timeout =
+          appTimeouts.get(ApplicationTimeoutType.LIFETIME);
       Assert.assertEquals("Application timeout string is incorrect.",
           formatISO8601, timeout.getExpiryTime());
       Assert.assertTrue("Application remaining time is incorrect",


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[43/50] [abbrv] hadoop git commit: HDFS-11140. Directory Scanner should log startup message time correctly. Contributed by Yiqun Lin.

Posted by st...@apache.org.
HDFS-11140. Directory Scanner should log startup message time correctly. Contributed by Yiqun Lin.


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

Branch: refs/heads/HADOOP-13345
Commit: 0857641f62778fad64e8158d78320efb0c8b417c
Parents: f54afdb
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 8 13:34:45 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 8 13:34:45 2016 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/RawLocalFileSystem.java   |  9 +++++++++
 .../hadoop/hdfs/server/datanode/DirectoryScanner.java   | 12 ++++++++----
 2 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0857641f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index 7bf429e..84863a5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -791,6 +791,15 @@ public class RawLocalFileSystem extends FileSystem {
           pathToFile(p).toPath(), BasicFileAttributeView.class);
       FileTime fmtime = (mtime >= 0) ? FileTime.fromMillis(mtime) : null;
       FileTime fatime = (atime >= 0) ? FileTime.fromMillis(atime) : null;
+
+      // On some macOS environment, BasicFileAttributeView.setTimes
+      // does not set times correctly when the argument of accessTime is null.
+      // TODO: Remove this after the issue is fixed.
+      if (fatime == null && Shell.MAC) {
+        FileStatus f = getFileStatus(p);
+        fatime = FileTime.fromMillis(f.getAccessTime());
+      }
+
       view.setTimes(fmtime, fatime, null);
     } catch (NoSuchFileException e) {
       throw new FileNotFoundException("File " + p + " does not exist");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0857641f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index e2baf32..18188dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -37,6 +37,8 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -61,7 +63,7 @@ public class DirectoryScanner implements Runnable {
   private static final int MILLIS_PER_SECOND = 1000;
   private static final String START_MESSAGE =
       "Periodic Directory Tree Verification scan"
-      + " starting at %dms with interval of %dms";
+      + " starting at %s with interval of %dms";
   private static final String START_MESSAGE_WITH_THROTTLE = START_MESSAGE
       + " and throttle limit of %dms/s";
 
@@ -268,10 +270,12 @@ public class DirectoryScanner implements Runnable {
     String logMsg;
 
     if (throttleLimitMsPerSec < MILLIS_PER_SECOND) {
-      logMsg = String.format(START_MESSAGE_WITH_THROTTLE, firstScanTime,
-          scanPeriodMsecs, throttleLimitMsPerSec);
+      logMsg = String.format(START_MESSAGE_WITH_THROTTLE,
+          FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs,
+          throttleLimitMsPerSec);
     } else {
-      logMsg = String.format(START_MESSAGE, firstScanTime, scanPeriodMsecs);
+      logMsg = String.format(START_MESSAGE,
+          FastDateFormat.getInstance().format(firstScanTime), scanPeriodMsecs);
     }
 
     LOG.info(logMsg);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[48/50] [abbrv] hadoop git commit: HADOOP-13867. FilterFileSystem should override rename(.., options) to take effect of Rename options called via FilterFileSystem implementations. Contributed By Vinayakumar B.

Posted by st...@apache.org.
HADOOP-13867. FilterFileSystem should override rename(.., options) to take effect of Rename options called via FilterFileSystem implementations. Contributed By Vinayakumar B.


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

Branch: refs/heads/HADOOP-13345
Commit: 0ef796174ecb5383f79cfecfcbfc4f309d093cd7
Parents: 4c2cf55
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Dec 8 18:57:43 2016 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Thu Dec 8 18:57:43 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FilterFileSystem.java     |  7 +++++++
 .../java/org/apache/hadoop/fs/TestFilterFileSystem.java | 12 +++++++++++-
 2 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ef79617/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
index 3f9aaa4..41429ac 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Progressable;
 
@@ -235,6 +236,12 @@ public class FilterFileSystem extends FileSystem {
   }
 
   @Override
+  protected void rename(Path src, Path dst, Rename... options)
+      throws IOException {
+    fs.rename(src, dst, options);
+  }
+
+  @Override
   public boolean truncate(Path f, final long newLength) throws IOException {
     return fs.truncate(f, newLength);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ef79617/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
index 24f3dc8..4cbb8ab 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
@@ -64,7 +64,6 @@ public class TestFilterFileSystem {
     public FSDataOutputStream append(Path f, int bufferSize) throws
         IOException;
     public long getLength(Path f);
-    public void rename(Path src, Path dst, Rename... options);
     public boolean exists(Path f);
     public boolean isDirectory(Path f);
     public boolean isFile(Path f);
@@ -264,6 +263,17 @@ public class TestFilterFileSystem {
     verify(mockFs).setWriteChecksum(eq(true));
   }
 
+  @Test
+  public void testRenameOptions() throws Exception {
+    FileSystem mockFs = mock(FileSystem.class);
+    FileSystem fs = new FilterFileSystem(mockFs);
+    Path src = new Path("/src");
+    Path dst = new Path("/dest");
+    Rename opt = Rename.TO_TRASH;
+    fs.rename(src, dst, opt);
+    verify(mockFs).rename(eq(src), eq(dst), eq(opt));
+  }
+
   private void checkInit(FilterFileSystem fs, boolean expectInit)
       throws Exception {
     URI uri = URI.create("filter:/");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/50] [abbrv] hadoop git commit: HADOOP-13859. TestConfigurationFieldsBase fails for fields that are DEFAULT values of skipped properties. (Haibo Chen via kasha)

Posted by st...@apache.org.
HADOOP-13859. TestConfigurationFieldsBase fails for fields that are DEFAULT values of skipped properties. (Haibo Chen via kasha)


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

Branch: refs/heads/HADOOP-13345
Commit: c8d0a049b00536385f06fad412a2288f005bf2ce
Parents: da4ecc9
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Tue Dec 6 14:08:08 2016 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Tue Dec 6 14:10:58 2016 -0800

----------------------------------------------------------------------
 .../conf/TestConfigurationFieldsBase.java       | 20 ++++++++++++++--
 .../hadoop/tools/TestHdfsConfigFields.java      |  6 -----
 .../yarn/conf/TestYarnConfigurationFields.java  | 24 --------------------
 3 files changed, 18 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8d0a049/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
index 9007c20..2367f21 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
@@ -205,6 +205,12 @@ public abstract class TestConfigurationFieldsBase {
       if (!f.getType().getName().equals("java.lang.String")) {
         continue;
       }
+
+      // filter out default-value fields
+      if (isFieldADefaultValue(f)) {
+        continue;
+      }
+
       // Convert found member into String
       try {
         value = (String) f.get(null);
@@ -332,6 +338,17 @@ public abstract class TestConfigurationFieldsBase {
   }
 
   /**
+   * Test if a field is a default value of another property by
+   * checking if its name starts with "DEFAULT_" or ends with
+   * "_DEFAULT".
+   * @param field the field to check
+   */
+  private static boolean isFieldADefaultValue(Field field) {
+    return field.getName().startsWith("DEFAULT_") ||
+        field.getName().endsWith("_DEFAULT");
+  }
+
+  /**
    * Utility function to extract &quot;public static final&quot; default
    * member variables from a Configuration type class.
    *
@@ -363,8 +380,7 @@ public abstract class TestConfigurationFieldsBase {
       }
       // Special: Stuff any property beginning with "DEFAULT_" into a
       // different hash for later processing
-      if (f.getName().startsWith("DEFAULT_") ||
-          f.getName().endsWith("_DEFAULT")) {
+      if (isFieldADefaultValue(f)) {
         if (retVal.containsKey(f.getName())) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8d0a049/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
index 3bbb609..a089d39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
@@ -57,12 +57,6 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
     // Remove deprecated properties listed in Configuration#DeprecationDelta
     configurationPropsToSkipCompare.add(DFSConfigKeys.DFS_DF_INTERVAL_KEY);
 
-    // Remove default properties
-    configurationPropsToSkipCompare
-        .add(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_DEFAULT);
-    configurationPropsToSkipCompare
-        .add(DFSConfigKeys.DFS_WEBHDFS_AUTHENTICATION_FILTER_DEFAULT);
-
     // Remove support property
     configurationPropsToSkipCompare
         .add(DFSConfigKeys.DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8d0a049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index db9364a..3da4bab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -49,22 +49,6 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
     // Specific properties to skip
     configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_FS_NODE_LABELS_STORE_IMPL_CLASS);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_RM_CONFIGURATION_PROVIDER_CLASS);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_CLIENT_FAILOVER_PROXY_PROVIDER);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_IPC_RECORD_FACTORY_CLASS);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_IPC_CLIENT_FACTORY_CLASS);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_IPC_SERVER_FACTORY_CLASS);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_IPC_RPC_IMPL);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_RM_SCHEDULER);
-    configurationPropsToSkipCompare
         .add(YarnConfiguration
             .YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONCLIENT_PROTOCOL);
     configurationPropsToSkipCompare
@@ -82,14 +66,6 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration
             .YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_SCM_STORE_CLASS);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_SCM_APP_CHECKER_CLASS);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_SHARED_CACHE_CHECKSUM_ALGO_IMPL);
-    configurationPropsToSkipCompare
-        .add(YarnConfiguration.DEFAULT_AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
 
     // Ignore blacklisting nodes for AM failures feature since it is still a


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org