Browse Source

Merge trunk into HDFS-7240

Arpit Agarwal 9 years ago
parent
commit
78bd1b2ab2
71 changed files with 1108 additions and 5272 deletions
  1. 150 0
      dev-support/verify-xml.sh
  2. 7 0
      hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
  3. 0 75
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-metrics.properties
  4. 0 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  5. 0 214
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ContextFactory.java
  6. 0 125
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsContext.java
  7. 0 49
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsException.java
  8. 0 254
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsRecord.java
  9. 0 188
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsServlet.java
  10. 0 104
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsUtil.java
  11. 0 41
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/Updater.java
  12. 0 43
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/file/package.html
  13. 0 276
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java
  14. 0 147
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
  15. 0 80
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/package.html
  16. 0 36
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/jvm/EventCounter.java
  17. 0 203
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/jvm/JvmMetrics.java
  18. 0 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/jvm/package-info.java
  19. 0 159
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/package.html
  20. 0 494
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java
  21. 0 206
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/CompositeContext.java
  22. 0 58
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/MetricValue.java
  23. 0 304
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/MetricsRecordImpl.java
  24. 0 61
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/NoEmitMetricsContext.java
  25. 0 74
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/NullContext.java
  26. 0 82
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/NullContextWithUpdateThread.java
  27. 0 93
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/OutputRecord.java
  28. 0 68
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/Util.java
  29. 0 36
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/package.html
  30. 0 92
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MBeanUtil.java
  31. 0 51
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsBase.java
  32. 0 229
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsDynamicMBeanBase.java
  33. 0 106
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsIntValue.java
  34. 0 93
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsLongValue.java
  35. 0 90
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsRegistry.java
  36. 0 129
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingInt.java
  37. 0 125
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingLong.java
  38. 0 198
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingRate.java
  39. 0 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/package-info.java
  40. 47 42
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
  41. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
  42. 255 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java
  43. 5 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  44. 0 112
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics/TestMetricsServlet.java
  45. 0 84
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics/ganglia/TestGangliaContext.java
  46. 0 39
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics/spi/TestOutputRecord.java
  47. 33 29
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
  48. 3 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  49. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
  50. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  51. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  52. 38 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  53. 35 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java
  54. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
  55. 31 63
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetrics.java
  56. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
  57. 37 54
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleClientMetrics.java
  58. 18 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
  59. 23 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
  60. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  61. 28 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
  62. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
  63. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
  64. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
  65. 27 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java
  66. 125 131
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
  67. 16 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
  68. 20 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
  69. 160 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStoreMetrics.java
  70. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LogInfo.java
  71. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java

+ 150 - 0
dev-support/verify-xml.sh

@@ -0,0 +1,150 @@
+#!/bin/bash
+##
+# 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.
+##
+# Script to run unit tests for xml <-> 1 or more Configuration file verification
+# usage: ./verify-xml.sh <mode>
+#
+
+# Utility functions
+function find_test_output_file() {
+  echo "Found test output file(s) at"
+  echo ""
+  if [ -n "$1" ] && [ -e "$1" ] ; then
+    echo "  $1"
+  fi
+  if [ -n "$2" ] && [ -e "$2" ] ; then
+    echo "  $2"
+  fi
+  if [ -n "$3" ] && [ -e "$3" ] ; then
+    echo "  $3"
+  fi
+  if [ -n "$4" ] && [ -e "$4" ] ; then
+    echo "  $4"
+  fi
+  echo ""
+  echo "Examine the file for specific information xml/Configuration mismatches."
+  echo ""
+}
+
+function print_test_banner() {
+  local banner_text=$1
+  local banner_length=${#banner_text}
+  local banner
+  banner=$( printf "%${banner_length}s" ' ' )
+  echo ""
+  echo "${banner// /=}"
+  echo "${banner_text}"
+  echo "${banner// /=}"
+  echo ""
+}
+
+# Wrapper functions for running unit tests
+function run_all_xml_test() {
+  mvn test -Dtest=TestCommonConfigurationFields,TestHdfsConfigFields,TestMapreduceConfigFields,TestYarnConfigurationFields
+  if [ $? -ne 0 ] ; then
+    print_test_banner "All Test*ConfigFields FAIL"
+  else
+    print_test_banner "All Test*ConfigFields SUCCESS"
+  fi
+}
+
+function run_common_xml_test() {
+  mvn test -Dtest=TestCommonConfigFields
+  if [ $? -ne 0 ] ; then
+    print_test_banner "TestCommonConfigurationFields FAIL"
+  else
+    print_test_banner "TestCommonConfigurationFields SUCCESS"
+  fi
+}
+
+function run_hdfs_xml_test() {
+  mvn test -Dtest=TestHdfsConfigFields
+  if [ $? -ne 0 ] ; then
+    print_test_banner "TestHdfsConfigFields FAIL"
+  else
+    print_test_banner "TestHdfsConfigFields SUCCESS"
+  fi
+}
+
+function run_mapreduce_xml_test() {
+  mvn test -Dtest=TestMapreduceConfigFields
+  if [ $? -ne 0 ] ; then
+    print_test_banner "TestMapreduceConfigFields FAIL"
+  else
+    print_test_banner "TestMapreduceConfigFields SUCCESS"
+  fi
+}
+
+function run_yarn_xml_test() {
+  mvn test -Dtest=TestYarnConfigurationFields
+  if [ $? -ne 0 ] ; then
+    print_test_banner "TestYarnConfigurationFields FAIL"
+  else
+    print_test_banner "TestYarnConfigurationFields SUCCESS"
+  fi
+}
+
+# Main body
+cd -P -- "$(dirname -- "${BASH_SOURCE-$0}")/.." || exit
+dir="$(pwd -P)"
+
+# - Create unit test file names
+export commonOutputFile
+commonOutputFile="$(find "${dir}" -name org.apache.hadoop.conf.TestCommonConfigurationFields-output.txt)"
+export hdfsOutputFile
+hdfsOutputFile="$(find "${dir}" -name org.apache.hadoop.tools.TestHdfsConfigFields-output.txt)"
+export mrOutputFile
+mrOutputFile="$(find "${dir}" -name org.apache.hadoop.mapreduce.TestMapreduceConfigFields-output.txt)"
+export yarnOutputFile
+yarnOutputFile="$(find "${dir}" -name org.apache.hadoop.yarn.conf.TestYarnConfigurationFields-output.txt)"
+
+# - Determine which tests to run
+case "$1" in
+
+  all)
+    run_all_xml_test
+    find_test_output_file "${commonOutputFile}" "${hdfsOutputFile}" "${mrOutputFile}" "${yarnOutputFile}"
+    ;;
+
+  common)
+    run_common_xml_test
+    find_test_output_file "${commonOutputFile}"
+    ;;
+
+  hdfs)
+    run_hdfs_xml_test
+    find_test_output_file "${hdfsOutputFile}"
+    ;;
+
+  mr)
+    run_mapreduce_xml_test
+    find_test_output_file "${mrOutputFile}"
+    ;;
+
+  yarn)
+    run_yarn_xml_test
+    find_test_output_file "${yarnOutputFile}"
+    ;;
+
+  *)
+    echo "Usage: $0 <mode>"
+    echo "  where <mode> is one of all, common, hdfs, mr, yarn"
+    exit 1
+    ;;
+
+esac

+ 7 - 0
hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml

@@ -44,6 +44,13 @@
       </includes>
       <outputDirectory>lib/native</outputDirectory>
     </fileSet>
+    <fileSet>
+      <directory>../hadoop-pipes/target/native/examples</directory>
+      <includes>
+        <include>*</include>
+      </includes>
+      <outputDirectory>lib/native/examples</outputDirectory>
+    </fileSet>
     <fileSet>
       <directory>../hadoop-archives/target</directory>
       <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>

+ 0 - 75
hadoop-common-project/hadoop-common/src/main/conf/hadoop-metrics.properties

@@ -1,75 +0,0 @@
-# Configuration of the "dfs" context for null
-dfs.class=org.apache.hadoop.metrics.spi.NullContext
-
-# Configuration of the "dfs" context for file
-#dfs.class=org.apache.hadoop.metrics.file.FileContext
-#dfs.period=10
-#dfs.fileName=/tmp/dfsmetrics.log
-
-# Configuration of the "dfs" context for ganglia
-# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
-# dfs.class=org.apache.hadoop.metrics.ganglia.GangliaContext
-# dfs.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
-# dfs.period=10
-# dfs.servers=localhost:8649
-
-
-# Configuration of the "mapred" context for null
-mapred.class=org.apache.hadoop.metrics.spi.NullContext
-
-# Configuration of the "mapred" context for file
-#mapred.class=org.apache.hadoop.metrics.file.FileContext
-#mapred.period=10
-#mapred.fileName=/tmp/mrmetrics.log
-
-# Configuration of the "mapred" context for ganglia
-# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
-# mapred.class=org.apache.hadoop.metrics.ganglia.GangliaContext
-# mapred.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
-# mapred.period=10
-# mapred.servers=localhost:8649
-
-
-# Configuration of the "jvm" context for null
-#jvm.class=org.apache.hadoop.metrics.spi.NullContext
-
-# Configuration of the "jvm" context for file
-#jvm.class=org.apache.hadoop.metrics.file.FileContext
-#jvm.period=10
-#jvm.fileName=/tmp/jvmmetrics.log
-
-# Configuration of the "jvm" context for ganglia
-# jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext
-# jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
-# jvm.period=10
-# jvm.servers=localhost:8649
-
-# Configuration of the "rpc" context for null
-rpc.class=org.apache.hadoop.metrics.spi.NullContext
-
-# Configuration of the "rpc" context for file
-#rpc.class=org.apache.hadoop.metrics.file.FileContext
-#rpc.period=10
-#rpc.fileName=/tmp/rpcmetrics.log
-
-# Configuration of the "rpc" context for ganglia
-# rpc.class=org.apache.hadoop.metrics.ganglia.GangliaContext
-# rpc.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
-# rpc.period=10
-# rpc.servers=localhost:8649
-
-
-# Configuration of the "ugi" context for null
-ugi.class=org.apache.hadoop.metrics.spi.NullContext
-
-# Configuration of the "ugi" context for file
-#ugi.class=org.apache.hadoop.metrics.file.FileContext
-#ugi.period=10
-#ugi.fileName=/tmp/ugimetrics.log
-
-# Configuration of the "ugi" context for ganglia
-# ugi.class=org.apache.hadoop.metrics.ganglia.GangliaContext
-# ugi.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
-# ugi.period=10
-# ugi.servers=localhost:8649
-

+ 0 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java

@@ -59,7 +59,6 @@ import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
 import org.apache.hadoop.security.ssl.SslSocketConnectorSecure;
 import org.apache.hadoop.jmx.JMXJsonServlet;
 import org.apache.hadoop.log.LogLevel;
-import org.apache.hadoop.metrics.MetricsServlet;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
@@ -572,14 +571,11 @@ public final class HttpServer2 implements FilterContainer {
 
   /**
    * Add default servlets.
-   * Note: /metrics servlet will be removed in 3.X release.
    */
-  @SuppressWarnings("deprecation")
   protected void addDefaultServlets() {
     // set up default servlets
     addServlet("stacks", "/stacks", StackServlet.class);
     addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
-    addServlet("metrics", "/metrics", MetricsServlet.class);
     addServlet("jmx", "/jmx", JMXJsonServlet.class);
     addServlet("conf", "/conf", ConfServlet.class);
   }

+ 0 - 214
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ContextFactory.java

@@ -1,214 +0,0 @@
-/*
- * ContextFactory.java
- *
- * 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.metrics;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.spi.NullContext;
-
-/**
- * Factory class for creating MetricsContext objects.  To obtain an instance
- * of this class, use the static <code>getFactory()</code> method.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Evolving
-public class ContextFactory {
-    
-  private static final String PROPERTIES_FILE = 
-    "/hadoop-metrics.properties";
-  private static final String CONTEXT_CLASS_SUFFIX =
-    ".class";
-  private static final String DEFAULT_CONTEXT_CLASSNAME =
-    "org.apache.hadoop.metrics.spi.NullContext";
-    
-  private static ContextFactory theFactory = null;
-    
-  private Map<String,Object> attributeMap = new HashMap<String,Object>();
-  private Map<String,MetricsContext> contextMap = 
-    new HashMap<String,MetricsContext>();
-    
-  // Used only when contexts, or the ContextFactory itself, cannot be
-  // created.
-  private static Map<String,MetricsContext> nullContextMap = 
-    new HashMap<String,MetricsContext>();
-    
-  /** Creates a new instance of ContextFactory */
-  protected ContextFactory() {
-  }
-    
-  /**
-   * Returns the value of the named attribute, or null if there is no 
-   * attribute of that name.
-   *
-   * @param attributeName the attribute name
-   * @return the attribute value
-   */
-  public Object getAttribute(String attributeName) {
-    return attributeMap.get(attributeName);
-  }
-    
-  /**
-   * Returns the names of all the factory's attributes.
-   * 
-   * @return the attribute names
-   */
-  public String[] getAttributeNames() {
-    String[] result = new String[attributeMap.size()];
-    int i = 0;
-    // for (String attributeName : attributeMap.keySet()) {
-    Iterator it = attributeMap.keySet().iterator();
-    while (it.hasNext()) {
-      result[i++] = (String) it.next();
-    }
-    return result;
-  }
-    
-  /**
-   * Sets the named factory attribute to the specified value, creating it
-   * if it did not already exist.  If the value is null, this is the same as
-   * calling removeAttribute.
-   *
-   * @param attributeName the attribute name
-   * @param value the new attribute value
-   */
-  public void setAttribute(String attributeName, Object value) {
-    attributeMap.put(attributeName, value);
-  }
-
-  /**
-   * Removes the named attribute if it exists.
-   *
-   * @param attributeName the attribute name
-   */
-  public void removeAttribute(String attributeName) {
-    attributeMap.remove(attributeName);
-  }
-    
-  /**
-   * Returns the named MetricsContext instance, constructing it if necessary 
-   * using the factory's current configuration attributes. <p/>
-   * 
-   * When constructing the instance, if the factory property 
-   * <i>contextName</i>.class</code> exists, 
-   * its value is taken to be the name of the class to instantiate.  Otherwise,
-   * the default is to create an instance of 
-   * <code>org.apache.hadoop.metrics.spi.NullContext</code>, which is a 
-   * dummy "no-op" context which will cause all metric data to be discarded.
-   * 
-   * @param contextName the name of the context
-   * @return the named MetricsContext
-   */
-  public synchronized MetricsContext getContext(String refName, String contextName)
-      throws IOException, ClassNotFoundException,
-             InstantiationException, IllegalAccessException {
-    MetricsContext metricsContext = contextMap.get(refName);
-    if (metricsContext == null) {
-      String classNameAttribute = refName + CONTEXT_CLASS_SUFFIX;
-      String className = (String) getAttribute(classNameAttribute);
-      if (className == null) {
-        className = DEFAULT_CONTEXT_CLASSNAME;
-      }
-      Class contextClass = Class.forName(className);
-      metricsContext = (MetricsContext) contextClass.newInstance();
-      metricsContext.init(contextName, this);
-      contextMap.put(contextName, metricsContext);
-    }
-    return metricsContext;
-  }
-
-  public synchronized MetricsContext getContext(String contextName)
-    throws IOException, ClassNotFoundException, InstantiationException,
-           IllegalAccessException {
-    return getContext(contextName, contextName);
-  }
-  
-  /** 
-   * Returns all MetricsContexts built by this factory.
-   */
-  public synchronized Collection<MetricsContext> getAllContexts() {
-    // Make a copy to avoid race conditions with creating new contexts.
-    return new ArrayList<MetricsContext>(contextMap.values());
-  }
-    
-  /**
-   * Returns a "null" context - one which does nothing.
-   */
-  public static synchronized MetricsContext getNullContext(String contextName) {
-    MetricsContext nullContext = nullContextMap.get(contextName);
-    if (nullContext == null) {
-      nullContext = new NullContext();
-      nullContextMap.put(contextName, nullContext);
-    }
-    return nullContext;
-  }
-    
-  /**
-   * Returns the singleton ContextFactory instance, constructing it if 
-   * necessary. <p/>
-   * 
-   * When the instance is constructed, this method checks if the file 
-   * <code>hadoop-metrics.properties</code> exists on the class path.  If it 
-   * exists, it must be in the format defined by java.util.Properties, and all 
-   * the properties in the file are set as attributes on the newly created
-   * ContextFactory instance.
-   *
-   * @return the singleton ContextFactory instance
-   */
-  public static synchronized ContextFactory getFactory() throws IOException {
-    if (theFactory == null) {
-      theFactory = new ContextFactory();
-      theFactory.setAttributes();
-    }
-    return theFactory;
-  }
-    
-  private void setAttributes() throws IOException {
-    InputStream is = getClass().getResourceAsStream(PROPERTIES_FILE);
-    if (is != null) {
-      try {
-        Properties properties = new Properties();
-        properties.load(is);
-        //for (Object propertyNameObj : properties.keySet()) {
-        Iterator it = properties.keySet().iterator();
-        while (it.hasNext()) {
-          String propertyName = (String) it.next();
-          String propertyValue = properties.getProperty(propertyName);
-          setAttribute(propertyName, propertyValue);
-        }
-      } finally {
-        is.close();
-      }
-    }
-  }
-    
-}

+ 0 - 125
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsContext.java

@@ -1,125 +0,0 @@
-/*
- * MetricsContext.java
- *
- * 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.metrics;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.spi.OutputRecord;
-
-/**
- * The main interface to the metrics package. 
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface MetricsContext {
-    
-  /**
-   * Default period in seconds at which data is sent to the metrics system.
-   */
-  public static final int DEFAULT_PERIOD = 5;
-
-  /**
-   * Initialize this context.
-   * @param contextName The given name for this context
-   * @param factory The creator of this context
-   */
-  public void init(String contextName, ContextFactory factory);
-
-  /**
-   * Returns the context name.
-   *
-   * @return the context name
-   */
-  public abstract String getContextName();
-    
-  /**
-   * Starts or restarts monitoring, the emitting of metrics records as they are 
-   * updated. 
-   */
-  public abstract void startMonitoring()
-    throws IOException;
-
-  /**
-   * Stops monitoring.  This does not free any data that the implementation
-   * may have buffered for sending at the next timer event. It
-   * is OK to call <code>startMonitoring()</code> again after calling 
-   * this.
-   * @see #close()
-   */
-  public abstract void stopMonitoring();
-    
-  /**
-   * Returns true if monitoring is currently in progress.
-   */
-  public abstract boolean isMonitoring();
-    
-  /**
-   * Stops monitoring and also frees any buffered data, returning this 
-   * object to its initial state.  
-   */
-  public abstract void close();
-    
-  /**
-   * Creates a new MetricsRecord instance with the given <code>recordName</code>.
-   * Throws an exception if the metrics implementation is configured with a fixed
-   * set of record names and <code>recordName</code> is not in that set.
-   *
-   * @param recordName the name of the record
-   * @throws MetricsException if recordName conflicts with configuration data
-   */
-  public abstract MetricsRecord createRecord(String recordName);
-    
-  /**
-   * Registers a callback to be called at regular time intervals, as 
-   * determined by the implementation-class specific configuration.
-   *
-   * @param updater object to be run periodically; it should updated
-   * some metrics records and then return
-   */
-  public abstract void registerUpdater(Updater updater);
-
-  /**
-   * Removes a callback, if it exists.
-   * 
-   * @param updater object to be removed from the callback list
-   */
-  public abstract void unregisterUpdater(Updater updater);
-  
-  /**
-   * Returns the timer period.
-   */
-  public abstract int getPeriod();
-  
-  /**
-   * Retrieves all the records managed by this MetricsContext.
-   * Useful for monitoring systems that are polling-based.
-   * 
-   * @return A non-null map from all record names to the records managed.
-   */
-   Map<String, Collection<OutputRecord>> getAllRecords();
-}

+ 0 - 49
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsException.java

@@ -1,49 +0,0 @@
-/*
- * MetricsException.java
- *
- * 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.metrics;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * General-purpose, unchecked metrics exception.
- * @deprecated Use {@link org.apache.hadoop.metrics2.MetricsException} instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Evolving
-public class MetricsException extends RuntimeException {
-    
-  private static final long serialVersionUID = -1643257498540498497L;
-
-  /** Creates a new instance of MetricsException */
-  public MetricsException() {
-  }
-    
-  /** Creates a new instance of MetricsException 
-   *
-   * @param message an error message
-   */
-  public MetricsException(String message) {
-    super(message);
-  }
-    
-}

+ 0 - 254
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsRecord.java

@@ -1,254 +0,0 @@
-/*
- * MetricsRecord.java
- *
- * 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.metrics;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * A named and optionally tagged set of records to be sent to the metrics
- * system. <p/>
- *
- * A record name identifies the kind of data to be reported. For example, a
- * program reporting statistics relating to the disks on a computer might use
- * a record name "diskStats".<p/>
- *
- * A record has zero or more <i>tags</i>. A tag has a name and a value. To
- * continue the example, the "diskStats" record might use a tag named
- * "diskName" to identify a particular disk.  Sometimes it is useful to have
- * more than one tag, so there might also be a "diskType" with value "ide" or
- * "scsi" or whatever.<p/>
- *
- * A record also has zero or more <i>metrics</i>.  These are the named
- * values that are to be reported to the metrics system.  In the "diskStats"
- * example, possible metric names would be "diskPercentFull", "diskPercentBusy", 
- * "kbReadPerSecond", etc.<p/>
- * 
- * The general procedure for using a MetricsRecord is to fill in its tag and
- * metric values, and then call <code>update()</code> to pass the record to the
- * client library.
- * Metric data is not immediately sent to the metrics system
- * each time that <code>update()</code> is called. 
- * An internal table is maintained, identified by the record name. This
- * table has columns 
- * corresponding to the tag and the metric names, and rows 
- * corresponding to each unique set of tag values. An update
- * either modifies an existing row in the table, or adds a new row with a set of
- * tag values that are different from all the other rows.  Note that if there
- * are no tags, then there can be at most one row in the table. <p/>
- * 
- * Once a row is added to the table, its data will be sent to the metrics system 
- * on every timer period, whether or not it has been updated since the previous
- * timer period.  If this is inappropriate, for example if metrics were being
- * reported by some transient object in an application, the <code>remove()</code>
- * method can be used to remove the row and thus stop the data from being
- * sent.<p/>
- *
- * Note that the <code>update()</code> method is atomic.  This means that it is
- * safe for different threads to be updating the same metric.  More precisely,
- * it is OK for different threads to call <code>update()</code> on MetricsRecord instances 
- * with the same set of tag names and tag values.  Different threads should 
- * <b>not</b> use the same MetricsRecord instance at the same time.
- *
- * @deprecated Use {@link org.apache.hadoop.metrics2.MetricsRecord} instead.
- */
-@Deprecated
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface MetricsRecord {
-    
-  /**
-   * Returns the record name. 
-   *
-   * @return the record name
-   */
-  public abstract String getRecordName();
-    
-  /**
-   * Sets the named tag to the specified value.  The tagValue may be null, 
-   * which is treated the same as an empty String.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  public abstract void setTag(String tagName, String tagValue);
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  public abstract void setTag(String tagName, int tagValue);
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  public abstract void setTag(String tagName, long tagValue);
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  public abstract void setTag(String tagName, short tagValue);
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  public abstract void setTag(String tagName, byte tagValue);
-    
-  /**
-   * Removes any tag of the specified name.
-   *
-   * @param tagName name of a tag
-   */
-  public abstract void removeTag(String tagName);
-  
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void setMetric(String metricName, int metricValue);
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void setMetric(String metricName, long metricValue);
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void setMetric(String metricName, short metricValue);
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void setMetric(String metricName, byte metricValue);
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void setMetric(String metricName, float metricValue);
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void incrMetric(String metricName, int metricValue);
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void incrMetric(String metricName, long metricValue);
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void incrMetric(String metricName, short metricValue);
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void incrMetric(String metricName, byte metricValue);
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  public abstract void incrMetric(String metricName, float metricValue);
-    
-  /**
-   * Updates the table of buffered data which is to be sent periodically.
-   * If the tag values match an existing row, that row is updated; 
-   * otherwise, a new row is added.
-   */
-  public abstract void update();
-    
-  /**
-   * Removes, from the buffered data table, all rows having tags 
-   * that equal the tags that have been set on this record. For example,
-   * if there are no tags on this record, all rows for this record name
-   * would be removed.  Or, if there is a single tag on this record, then
-   * just rows containing a tag with the same name and value would be removed.
-   */
-  public abstract void remove();
-    
-}

+ 0 - 188
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsServlet.java

@@ -1,188 +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.metrics;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.metrics.spi.OutputRecord;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
-import org.mortbay.util.ajax.JSON;
-import org.mortbay.util.ajax.JSON.Output;
-
-/**
- * A servlet to print out metrics data.  By default, the servlet returns a 
- * textual representation (no promises are made for parseability), and
- * users can use "?format=json" for parseable output.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class MetricsServlet extends HttpServlet {
-  
-  /**
-   * A helper class to hold a TagMap and MetricMap.
-   */
-  static class TagsMetricsPair implements JSON.Convertible {
-    final TagMap tagMap;
-    final MetricMap metricMap;
-    
-    public TagsMetricsPair(TagMap tagMap, MetricMap metricMap) {
-      this.tagMap = tagMap;
-      this.metricMap = metricMap;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void fromJSON(Map map) {
-      throw new UnsupportedOperationException();
-    }
-
-    /** Converts to JSON by providing an array. */
-    @Override
-    public void toJSON(Output out) {
-      out.add(new Object[] { tagMap, metricMap });
-    }
-  }
-  
-  /**
-   * Collects all metric data, and returns a map:
-   *   contextName -> recordName -> [ (tag->tagValue), (metric->metricValue) ].
-   * The values are either String or Number.  The final value is implemented
-   * as a list of TagsMetricsPair.
-   */
-   Map<String, Map<String, List<TagsMetricsPair>>> makeMap(
-       Collection<MetricsContext> contexts) throws IOException {
-    Map<String, Map<String, List<TagsMetricsPair>>> map = 
-      new TreeMap<String, Map<String, List<TagsMetricsPair>>>();
-
-    for (MetricsContext context : contexts) {
-      Map<String, List<TagsMetricsPair>> records = 
-        new TreeMap<String, List<TagsMetricsPair>>();
-      map.put(context.getContextName(), records);
-    
-      for (Map.Entry<String, Collection<OutputRecord>> r : 
-          context.getAllRecords().entrySet()) {
-        List<TagsMetricsPair> metricsAndTags = 
-          new ArrayList<TagsMetricsPair>();
-        records.put(r.getKey(), metricsAndTags);
-        for (OutputRecord outputRecord : r.getValue()) {
-          TagMap tagMap = outputRecord.getTagsCopy();
-          MetricMap metricMap = outputRecord.getMetricsCopy();
-          metricsAndTags.add(new TagsMetricsPair(tagMap, metricMap));
-        }
-      }
-    }
-    return map;
-  }
-  
-  @Override
-  public void doGet(HttpServletRequest request, HttpServletResponse response)
-      throws ServletException, IOException {
-
-    if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
-                                                   request, response)) {
-      return;
-    }
-
-    String format = request.getParameter("format");
-    Collection<MetricsContext> allContexts = 
-      ContextFactory.getFactory().getAllContexts();
-    if ("json".equals(format)) {
-      response.setContentType("application/json; charset=utf-8");
-      PrintWriter out = response.getWriter();
-      try {
-        // Uses Jetty's built-in JSON support to convert the map into JSON.
-        out.print(new JSON().toJSON(makeMap(allContexts)));
-      } finally {
-        out.close();
-      }
-    } else {
-      PrintWriter out = response.getWriter();
-      try {
-        printMap(out, makeMap(allContexts));
-      } finally {
-        out.close();
-      }
-    }
-  }
-  
-  /**
-   * Prints metrics data in a multi-line text form.
-   */
-  void printMap(PrintWriter out, Map<String, Map<String, List<TagsMetricsPair>>> map) {
-    for (Map.Entry<String, Map<String, List<TagsMetricsPair>>> context : map.entrySet()) {
-      out.print(context.getKey());
-      out.print("\n");
-      for (Map.Entry<String, List<TagsMetricsPair>> record : context.getValue().entrySet()) {
-        indent(out, 1);
-        out.print(record.getKey());
-        out.print("\n");
-        for (TagsMetricsPair pair : record.getValue()) {
-          indent(out, 2);
-          // Prints tag values in the form "{key=value,key=value}:"
-          out.print("{");
-          boolean first = true;
-          for (Map.Entry<String, Object> tagValue : pair.tagMap.entrySet()) {
-            if (first) {
-              first = false;
-            } else {
-              out.print(",");
-            }
-            out.print(tagValue.getKey());
-            out.print("=");
-            out.print(tagValue.getValue().toString());
-          }
-          out.print("}:\n");
-          
-          // Now print metric values, one per line
-          for (Map.Entry<String, Number> metricValue : 
-              pair.metricMap.entrySet()) {
-            indent(out, 3);
-            out.print(metricValue.getKey());
-            out.print("=");
-            out.print(metricValue.getValue().toString());
-            out.print("\n");
-          }
-        }
-      }
-    }    
-  }
-  
-  private void indent(PrintWriter out, int indent) {
-    for (int i = 0; i < indent; ++i) {
-      out.append("  ");
-    }
-  }
-}

+ 0 - 104
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsUtil.java

@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Utility class to simplify creation and reporting of hadoop metrics.
- *
- * For examples of usage, see NameNodeMetrics.
- * @see org.apache.hadoop.metrics.MetricsRecord
- * @see org.apache.hadoop.metrics.MetricsContext
- * @see org.apache.hadoop.metrics.ContextFactory
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Evolving
-public class MetricsUtil {
-    
-  public static final Log LOG =
-    LogFactory.getLog(MetricsUtil.class);
-
-  /**
-   * Don't allow creation of a new instance of Metrics
-   */
-  private MetricsUtil() {}
-    
-  public static MetricsContext getContext(String contextName) {
-    return getContext(contextName, contextName);
-  }
-
-  /**
-   * Utility method to return the named context.
-   * If the desired context cannot be created for any reason, the exception
-   * is logged, and a null context is returned.
-   */
-  public static MetricsContext getContext(String refName, String contextName) {
-    MetricsContext metricsContext;
-    try {
-      metricsContext =
-        ContextFactory.getFactory().getContext(refName, contextName);
-      if (!metricsContext.isMonitoring()) {
-        metricsContext.startMonitoring();
-      }
-    } catch (Exception ex) {
-      LOG.error("Unable to create metrics context " + contextName, ex);
-      metricsContext = ContextFactory.getNullContext(contextName);
-    }
-    return metricsContext;
-  }
-
-  /**
-   * Utility method to create and return new metrics record instance within the
-   * given context. This record is tagged with the host name.
-   *
-   * @param context the context
-   * @param recordName name of the record
-   * @return newly created metrics record
-   */
-  public static MetricsRecord createRecord(MetricsContext context, 
-                                           String recordName) 
-  {
-    MetricsRecord metricsRecord = context.createRecord(recordName);
-    metricsRecord.setTag("hostName", getHostName());
-    return metricsRecord;        
-  }
-    
-  /**
-   * Returns the host name.  If the host name is unobtainable, logs the
-   * exception and returns "unknown".
-   */
-  private static String getHostName() {
-    String hostName = null;
-    try {
-      hostName = InetAddress.getLocalHost().getHostName();
-    } catch (UnknownHostException ex) {
-      LOG.info("Unable to obtain hostName", ex);
-      hostName = "unknown";
-    }
-    return hostName;
-  }
-}

+ 0 - 41
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/Updater.java

@@ -1,41 +0,0 @@
-/*
- * Updater.java
- *
- * 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.metrics;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Call-back interface.  See <code>MetricsContext.registerUpdater()</code>.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Evolving
-public interface Updater {
-    
-  /**
-   * Timer-based call-back from the metric library. 
-   */
-  public abstract void doUpdates(MetricsContext context);
-
-}

+ 0 - 43
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/file/package.html

@@ -1,43 +0,0 @@
-<html>
-
-<!--
-   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.
--->
-
-<body>
-Implementation of the metrics package that writes the metrics to a file.
-Programmers should not normally need to use this package directly. Instead
-they should use org.hadoop.metrics.
-
-<p/>
-These are the implementation specific factory attributes 
-(See ContextFactory.getFactory()):
-
-<dl>
-    <dt><i>contextName</i>.fileName</dt>
-    <dd>The path of the file to which metrics in context <i>contextName</i>
-    are to be appended.  If this attribute is not specified, the metrics
-    are written to standard output by default.</dd>
-    
-    <dt><i>contextName</i>.period</dt>
-    <dd>The period in seconds on which the metric data is written to the
-    file.</dd>
-    
-</dl>
-
-
-</body>
-</html>

+ 0 - 276
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java

@@ -1,276 +0,0 @@
-/*
- * GangliaContext.java
- *
- * 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.metrics.ganglia;
-
-import java.io.IOException;
-import java.net.*;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.io.Charsets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext;
-import org.apache.hadoop.metrics.spi.OutputRecord;
-import org.apache.hadoop.metrics.spi.Util;
-
-/**
- * Context for sending metrics to Ganglia.
- *
- * @deprecated Use {@link org.apache.hadoop.metrics2.sink.ganglia.GangliaSink30}
- * instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class GangliaContext extends AbstractMetricsContext {
-    
-  private static final String PERIOD_PROPERTY = "period";
-  private static final String SERVERS_PROPERTY = "servers";
-  private static final String UNITS_PROPERTY = "units";
-  private static final String SLOPE_PROPERTY = "slope";
-  private static final String TMAX_PROPERTY = "tmax";
-  private static final String DMAX_PROPERTY = "dmax";
-  private static final String MULTICAST_PROPERTY = "multicast";
-  private static final String MULTICAST_TTL_PROPERTY = "multicast.ttl";
-
-  private static final String DEFAULT_UNITS = "";
-  private static final String DEFAULT_SLOPE = "both";
-  private static final int DEFAULT_TMAX = 60;
-  private static final int DEFAULT_DMAX = 0;
-  private static final int DEFAULT_PORT = 8649;
-  private static final int BUFFER_SIZE = 1500;       // as per libgmond.c
-  private static final int DEFAULT_MULTICAST_TTL = 1;
-
-  private final Log LOG = LogFactory.getLog(this.getClass());    
-
-  private static final Map<Class,String> typeTable = new HashMap<Class,String>(5);
-    
-  static {
-    typeTable.put(String.class, "string");
-    typeTable.put(Byte.class, "int8");
-    typeTable.put(Short.class, "int16");
-    typeTable.put(Integer.class, "int32");
-    typeTable.put(Long.class, "float");
-    typeTable.put(Float.class, "float");
-  }
-    
-  protected byte[] buffer = new byte[BUFFER_SIZE];
-  protected int offset;
-    
-  protected List<? extends SocketAddress> metricsServers;
-  private Map<String,String> unitsTable;
-  private Map<String,String> slopeTable;
-  private Map<String,String> tmaxTable;
-  private Map<String,String> dmaxTable;
-  private boolean multicastEnabled;
-  private int multicastTtl;
-    
-  protected DatagramSocket datagramSocket;
-    
-  /** Creates a new instance of GangliaContext */
-  @InterfaceAudience.Private
-  public GangliaContext() {
-  }
-    
-  @Override
-  @InterfaceAudience.Private
-  public void init(String contextName, ContextFactory factory) {
-    super.init(contextName, factory);
-    parseAndSetPeriod(PERIOD_PROPERTY);
-        
-    metricsServers = 
-      Util.parse(getAttribute(SERVERS_PROPERTY), DEFAULT_PORT); 
-        
-    unitsTable = getAttributeTable(UNITS_PROPERTY);
-    slopeTable = getAttributeTable(SLOPE_PROPERTY);
-    tmaxTable  = getAttributeTable(TMAX_PROPERTY);
-    dmaxTable  = getAttributeTable(DMAX_PROPERTY);
-    multicastEnabled = Boolean.parseBoolean(getAttribute(MULTICAST_PROPERTY));
-    String multicastTtlValue = getAttribute(MULTICAST_TTL_PROPERTY);
-    if (multicastEnabled) {
-      if (multicastTtlValue == null) {
-        multicastTtl = DEFAULT_MULTICAST_TTL;
-      } else {
-        multicastTtl = Integer.parseInt(multicastTtlValue);
-      }
-    }
-        
-    try {
-      if (multicastEnabled) {
-        LOG.info("Enabling multicast for Ganglia with TTL " + multicastTtl);
-        datagramSocket = new MulticastSocket();
-        ((MulticastSocket) datagramSocket).setTimeToLive(multicastTtl);
-      } else {
-        datagramSocket = new DatagramSocket();
-      }
-    } catch (IOException e) {
-      LOG.error(e);
-    }
-  }
-
-    /**
-   * method to close the datagram socket
-   */
-  @Override
-  public void close() {
-    super.close();
-    if (datagramSocket != null) {
-      datagramSocket.close();
-    }
-  }
-  
-  @Override
-  @InterfaceAudience.Private
-  public void emitRecord(String contextName, String recordName,
-    OutputRecord outRec) 
-  throws IOException {
-    // Setup so that the records have the proper leader names so they are
-    // unambiguous at the ganglia level, and this prevents a lot of rework
-    StringBuilder sb = new StringBuilder();
-    sb.append(contextName);
-    sb.append('.');
-
-    if (contextName.equals("jvm") && outRec.getTag("processName") != null) {
-      sb.append(outRec.getTag("processName"));
-      sb.append('.');
-    }
-
-    sb.append(recordName);
-    sb.append('.');
-    int sbBaseLen = sb.length();
-
-    // emit each metric in turn
-    for (String metricName : outRec.getMetricNames()) {
-      Object metric = outRec.getMetric(metricName);
-      String type = typeTable.get(metric.getClass());
-      if (type != null) {
-        sb.append(metricName);
-        emitMetric(sb.toString(), type, metric.toString());
-        sb.setLength(sbBaseLen);
-      } else {
-        LOG.warn("Unknown metrics type: " + metric.getClass());
-      }
-    }
-  }
-    
-  protected void emitMetric(String name, String type,  String value) 
-  throws IOException {
-    String units = getUnits(name);
-    int slope = getSlope(name);
-    int tmax = getTmax(name);
-    int dmax = getDmax(name);
-        
-    offset = 0;
-    xdr_int(0);             // metric_user_defined
-    xdr_string(type);
-    xdr_string(name);
-    xdr_string(value);
-    xdr_string(units);
-    xdr_int(slope);
-    xdr_int(tmax);
-    xdr_int(dmax);
-        
-    for (SocketAddress socketAddress : metricsServers) {
-      DatagramPacket packet = 
-        new DatagramPacket(buffer, offset, socketAddress);
-      datagramSocket.send(packet);
-    }
-  }
-    
-  protected String getUnits(String metricName) {
-    String result = unitsTable.get(metricName);
-    if (result == null) {
-      result = DEFAULT_UNITS;
-    }
-    return result;
-  }
-    
-  protected int getSlope(String metricName) {
-    String slopeString = slopeTable.get(metricName);
-    if (slopeString == null) {
-      slopeString = DEFAULT_SLOPE; 
-    }
-    return ("zero".equals(slopeString) ? 0 : 3); // see gmetric.c
-  }
-    
-  protected int getTmax(String metricName) {
-    if (tmaxTable == null) {
-      return DEFAULT_TMAX;
-    }
-    String tmaxString = tmaxTable.get(metricName);
-    if (tmaxString == null) {
-      return DEFAULT_TMAX;
-    }
-    else {
-      return Integer.parseInt(tmaxString);
-    }
-  }
-    
-  protected int getDmax(String metricName) {
-    String dmaxString = dmaxTable.get(metricName);
-    if (dmaxString == null) {
-      return DEFAULT_DMAX;
-    }
-    else {
-      return Integer.parseInt(dmaxString);
-    }
-  }
-    
-  /**
-   * Puts a string into the buffer by first writing the size of the string
-   * as an int, followed by the bytes of the string, padded if necessary to
-   * a multiple of 4.
-   */
-  protected void xdr_string(String s) {
-    byte[] bytes = s.getBytes(Charsets.UTF_8);
-    int len = bytes.length;
-    xdr_int(len);
-    System.arraycopy(bytes, 0, buffer, offset, len);
-    offset += len;
-    pad();
-  }
-
-  /**
-   * Pads the buffer with zero bytes up to the nearest multiple of 4.
-   */
-  private void pad() {
-    int newOffset = ((offset + 3) / 4) * 4;
-    while (offset < newOffset) {
-      buffer[offset++] = 0;
-    }
-  }
-        
-  /**
-   * Puts an integer into the buffer as 4 bytes, big-endian.
-   */
-  protected void xdr_int(int i) {
-    buffer[offset++] = (byte)((i >> 24) & 0xff);
-    buffer[offset++] = (byte)((i >> 16) & 0xff);
-    buffer[offset++] = (byte)((i >> 8) & 0xff);
-    buffer[offset++] = (byte)(i & 0xff);
-  }
-}

+ 0 - 147
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java

@@ -1,147 +0,0 @@
-/*
- * GangliaContext.java
- *
- * 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.metrics.ganglia;
-
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.SocketAddress;
-import java.net.UnknownHostException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.net.DNS;
-
-/**
- * Context for sending metrics to Ganglia version 3.1.x.
- * 
- * 3.1.1 has a slightly different wire portal compared to 3.0.x.
- *
- * @deprecated Use {@link org.apache.hadoop.metrics2.sink.ganglia.GangliaSink31}
- * instead.
- */
-@Deprecated
-public class GangliaContext31 extends GangliaContext {
-
-  String hostName = "UNKNOWN.example.com";
-
-  private static final Log LOG = 
-    LogFactory.getLog("org.apache.hadoop.util.GangliaContext31");
-
-  @Override
-  public void init(String contextName, ContextFactory factory) {
-    super.init(contextName, factory);
-
-    LOG.debug("Initializing the GangliaContext31 for Ganglia 3.1 metrics.");
-
-    // Take the hostname from the DNS class.
-
-    Configuration conf = new Configuration();
-
-    if (conf.get("slave.host.name") != null) {
-      hostName = conf.get("slave.host.name");
-    } else {
-      try {
-        hostName = DNS.getDefaultHost(
-          conf.get("dfs.datanode.dns.interface","default"),
-          conf.get("dfs.datanode.dns.nameserver","default"));
-      } catch (UnknownHostException uhe) {
-        LOG.error(uhe);
-    	hostName = "UNKNOWN.example.com";
-      }
-    }
-  }
-
-  @Override
-  protected void emitMetric(String name, String type,  String value) 
-    throws IOException
-  {
-    if (name == null) {
-      LOG.warn("Metric was emitted with no name.");
-      return;
-    } else if (value == null) {
-      LOG.warn("Metric name " + name +" was emitted with a null value.");
-      return;
-    } else if (type == null) {
-      LOG.warn("Metric name " + name + ", value " + value + " has no type.");
-      return;
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Emitting metric " + name + ", type " + type + ", value " +
-          value + " from hostname" + hostName);
-    }
-
-    String units = getUnits(name);
-    int slope = getSlope(name);
-    int tmax = getTmax(name);
-    int dmax = getDmax(name);
-    offset = 0;
-    String groupName = name.substring(0,name.lastIndexOf("."));
-
-    // The following XDR recipe was done through a careful reading of
-    // gm_protocol.x in Ganglia 3.1 and carefully examining the output of
-    // the gmetric utility with strace.
-
-    // First we send out a metadata message
-    xdr_int(128);         // metric_id = metadata_msg
-    xdr_string(hostName); // hostname
-    xdr_string(name);     // metric name
-    xdr_int(0);           // spoof = False
-    xdr_string(type);     // metric type
-    xdr_string(name);     // metric name
-    xdr_string(units);    // units
-    xdr_int(slope);       // slope
-    xdr_int(tmax);        // tmax, the maximum time between metrics
-    xdr_int(dmax);        // dmax, the maximum data value
-
-    xdr_int(1);             /*Num of the entries in extra_value field for 
-                              Ganglia 3.1.x*/
-    xdr_string("GROUP");    /*Group attribute*/
-    xdr_string(groupName);  /*Group value*/
-
-    for (SocketAddress socketAddress : metricsServers) {
-      DatagramPacket packet =
-        new DatagramPacket(buffer, offset, socketAddress);
-      datagramSocket.send(packet);
-    }
-
-    // Now we send out a message with the actual value.
-    // Technically, we only need to send out the metadata message once for
-    // each metric, but I don't want to have to record which metrics we did and
-    // did not send.
-    offset = 0;
-    xdr_int(133);         // we are sending a string value
-    xdr_string(hostName); // hostName
-    xdr_string(name);     // metric name
-    xdr_int(0);           // spoof = False
-    xdr_string("%s");     // format field
-    xdr_string(value);    // metric value
-        
-    for (SocketAddress socketAddress : metricsServers) {
-      DatagramPacket packet = 
-        new DatagramPacket(buffer, offset, socketAddress);
-      datagramSocket.send(packet);
-    }
-  }
-
-}

+ 0 - 80
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/package.html

@@ -1,80 +0,0 @@
-<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN">
-<html>
-
-<!--
-   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.
--->
-
-<body>
-<!--
- * 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.
--->
-
-Implementation of the metrics package that sends metric data to 
-<a href="http://ganglia.sourceforge.net/">Ganglia</a>.
-Programmers should not normally need to use this package directly. Instead
-they should use org.hadoop.metrics.
-
-<p/>
-These are the implementation specific factory attributes 
-(See ContextFactory.getFactory()):
-
-<dl>
-    <dt><i>contextName</i>.servers</dt>
-    <dd>Space and/or comma separated sequence of servers to which UDP
-    messages should be sent.</dd>
-    
-    <dt><i>contextName</i>.period</dt>
-    <dd>The period in seconds on which the metric data is sent to the
-    server(s).</dd>
-
-    <dt><i>contextName</i>.multicast</dt>
-    <dd>Enable multicast for Ganglia</dd>
-
-    <dt><i>contextName</i>.multicast.ttl</dt>
-    <dd>TTL for multicast packets</dd>
-    
-    <dt><i>contextName</i>.units.<i>recordName</i>.<i>metricName</i></dt>
-    <dd>The units for the specified metric in the specified record.</dd>
-    
-    <dt><i>contextName</i>.slope.<i>recordName</i>.<i>metricName</i></dt>
-    <dd>The slope for the specified metric in the specified record.</dd>
-    
-    <dt><i>contextName</i>.tmax.<i>recordName</i>.<i>metricName</i></dt>
-    <dd>The tmax for the specified metric in the specified record.</dd>
-    
-    <dt><i>contextName</i>.dmax.<i>recordName</i>.<i>metricName</i></dt>
-    <dd>The dmax for the specified metric in the specified record.</dd>
-    
-</dl>
-
-
-</body>
-</html>

+ 0 - 36
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/jvm/EventCounter.java

@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics.jvm;
-
-/**
- * A log4J Appender that simply counts logging events in three levels:
- * fatal, error and warn.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-public class EventCounter extends org.apache.hadoop.log.metrics.EventCounter {
-
-  static {
-    // The logging system is not started yet.
-    System.err.println("WARNING: "+ EventCounter.class.getName() +
-        " is deprecated. Please use "+
-        org.apache.hadoop.log.metrics.EventCounter.class.getName() +
-        " in all the log4j.properties files.");
-  }
-}

+ 0 - 203
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/jvm/JvmMetrics.java

@@ -1,203 +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.metrics.jvm;
-
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryMXBean;
-import java.lang.management.MemoryUsage;
-import java.lang.management.ThreadInfo;
-import java.lang.management.ThreadMXBean;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-
-import static java.lang.Thread.State.*;
-import java.lang.management.GarbageCollectorMXBean;
-import java.util.List;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * Singleton class which reports Java Virtual Machine metrics to the metrics API.  
- * Any application can create an instance of this class in order to emit
- * Java VM metrics.  
- *
- * @deprecated Use {@link org.apache.hadoop.metrics2.source.JvmMetrics} instead.
- */
-@Deprecated
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class JvmMetrics implements Updater {
-    
-    private static final float M = 1024*1024;
-    private static JvmMetrics theInstance = null;
-    private static Log log = LogFactory.getLog(JvmMetrics.class);
-    
-    private MetricsRecord metrics;
-    
-    // garbage collection counters
-    private long gcCount = 0;
-    private long gcTimeMillis = 0;
-    
-    // logging event counters
-    private long fatalCount = 0;
-    private long errorCount = 0;
-    private long warnCount  = 0;
-    private long infoCount  = 0;
-    
-    public synchronized static JvmMetrics init(String processName, String sessionId) {
-      return init(processName, sessionId, "metrics");
-    }
-    
-    public synchronized static JvmMetrics init(String processName, String sessionId,
-      String recordName) {
-        if (theInstance != null) {
-            log.info("Cannot initialize JVM Metrics with processName=" + 
-                     processName + ", sessionId=" + sessionId + 
-                     " - already initialized");
-        }
-        else {
-            log.info("Initializing JVM Metrics with processName=" 
-                    + processName + ", sessionId=" + sessionId);
-            theInstance = new JvmMetrics(processName, sessionId, recordName);
-        }
-        return theInstance;
-    }
-    
-    /** Creates a new instance of JvmMetrics */
-    private JvmMetrics(String processName, String sessionId,
-      String recordName) {
-        MetricsContext context = MetricsUtil.getContext("jvm");
-        metrics = MetricsUtil.createRecord(context, recordName);
-        metrics.setTag("processName", processName);
-        metrics.setTag("sessionId", sessionId);
-        context.registerUpdater(this);
-    }
-    
-    /**
-     * This will be called periodically (with the period being configuration
-     * dependent).
-     */
-    @Override
-    public void doUpdates(MetricsContext context) {
-        doMemoryUpdates();
-        doGarbageCollectionUpdates();
-        doThreadUpdates();
-        doEventCountUpdates();
-        metrics.update();
-    }
-    
-    private void doMemoryUpdates() {
-        MemoryMXBean memoryMXBean =
-               ManagementFactory.getMemoryMXBean();
-        MemoryUsage memNonHeap =
-                memoryMXBean.getNonHeapMemoryUsage();
-        MemoryUsage memHeap =
-                memoryMXBean.getHeapMemoryUsage();
-        Runtime runtime = Runtime.getRuntime();
-
-        metrics.setMetric("memNonHeapUsedM", memNonHeap.getUsed()/M);
-        metrics.setMetric("memNonHeapCommittedM", memNonHeap.getCommitted()/M);
-        metrics.setMetric("memHeapUsedM", memHeap.getUsed()/M);
-        metrics.setMetric("memHeapCommittedM", memHeap.getCommitted()/M);
-        metrics.setMetric("maxMemoryM", runtime.maxMemory()/M);
-    }
-    
-    private void doGarbageCollectionUpdates() {
-        List<GarbageCollectorMXBean> gcBeans =
-                ManagementFactory.getGarbageCollectorMXBeans();
-        long count = 0;
-        long timeMillis = 0;
-        for (GarbageCollectorMXBean gcBean : gcBeans) {
-            count += gcBean.getCollectionCount();
-            timeMillis += gcBean.getCollectionTime();
-        }
-        metrics.incrMetric("gcCount", (int)(count - gcCount));
-        metrics.incrMetric("gcTimeMillis", (int)(timeMillis - gcTimeMillis));
-        
-        gcCount = count;
-        gcTimeMillis = timeMillis;
-    }
-    
-    private void doThreadUpdates() {
-        ThreadMXBean threadMXBean =
-                ManagementFactory.getThreadMXBean();
-        long threadIds[] = 
-                threadMXBean.getAllThreadIds();
-        ThreadInfo[] threadInfos =
-                threadMXBean.getThreadInfo(threadIds, 0);
-        
-        int threadsNew = 0;
-        int threadsRunnable = 0;
-        int threadsBlocked = 0;
-        int threadsWaiting = 0;
-        int threadsTimedWaiting = 0;
-        int threadsTerminated = 0;
-        
-        for (ThreadInfo threadInfo : threadInfos) {
-            // threadInfo is null if the thread is not alive or doesn't exist
-            if (threadInfo == null) continue;
-            Thread.State state = threadInfo.getThreadState();
-            if (state == NEW) {
-                threadsNew++;
-            } 
-            else if (state == RUNNABLE) {
-                threadsRunnable++;
-            }
-            else if (state == BLOCKED) {
-                threadsBlocked++;
-            }
-            else if (state == WAITING) {
-                threadsWaiting++;
-            } 
-            else if (state == TIMED_WAITING) {
-                threadsTimedWaiting++;
-            }
-            else if (state == TERMINATED) {
-                threadsTerminated++;
-            }
-        }
-        metrics.setMetric("threadsNew", threadsNew);
-        metrics.setMetric("threadsRunnable", threadsRunnable);
-        metrics.setMetric("threadsBlocked", threadsBlocked);
-        metrics.setMetric("threadsWaiting", threadsWaiting);
-        metrics.setMetric("threadsTimedWaiting", threadsTimedWaiting);
-        metrics.setMetric("threadsTerminated", threadsTerminated);
-    }
-    
-    private void doEventCountUpdates() {
-        long newFatal = EventCounter.getFatal();
-        long newError = EventCounter.getError();
-        long newWarn  = EventCounter.getWarn();
-        long newInfo  = EventCounter.getInfo();
-        
-        metrics.incrMetric("logFatal", (int)(newFatal - fatalCount));
-        metrics.incrMetric("logError", (int)(newError - errorCount));
-        metrics.incrMetric("logWarn",  (int)(newWarn - warnCount));
-        metrics.incrMetric("logInfo",  (int)(newInfo - infoCount));
-        
-        fatalCount = newFatal;
-        errorCount = newError;
-        warnCount  = newWarn;
-        infoCount  = newInfo;
-    }
-}

+ 0 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/jvm/package-info.java

@@ -1,22 +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.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-package org.apache.hadoop.metrics.jvm;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;

+ 0 - 159
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/package.html

@@ -1,159 +0,0 @@
-<html>
-
-<!--
-   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.
--->
-
-  <head>
-    <title>org.apache.hadoop.metrics</title>
-  </head>
-<body>
-This package defines an API for reporting performance metric information.
-<p/>
-The API is abstract so that it can be implemented on top of
-a variety of metrics client libraries.  The choice of 
-client library is a configuration option, and different 
-modules within the same application can use
-different metrics implementation libraries.
-<p/>
-Sub-packages:
-<dl>
-    <dt><code>org.apache.hadoop.metrics.spi</code></dt>
-    <dd>The abstract Server Provider Interface package. Those wishing to
-    integrate the metrics API with a particular metrics client library should 
-    extend this package.</dd>
-    
-    <dt><code>org.apache.hadoop.metrics.file</code></dt>
-    <dd>An implementation package which writes the metric data to 
-    a file, or sends it to the standard output stream.</dd>
- 
-    <dt> <code>org.apache.hadoop.metrics.ganglia</code></dt>
-    <dd>An implementation package which sends metric data to 
-    <a href="http://ganglia.sourceforge.net/">Ganglia</a>.</dd>
-</dl>
-
-<h3>Introduction to the Metrics API</h3>
-
-Here is a simple example of how to use this package to report a single
-metric value:
-<pre>
-    private ContextFactory contextFactory = ContextFactory.getFactory();
-    
-    void reportMyMetric(float myMetric) {
-        MetricsContext myContext = contextFactory.getContext("myContext");
-        MetricsRecord myRecord = myContext.getRecord("myRecord");
-        myRecord.setMetric("myMetric", myMetric);
-        myRecord.update();
-    }
-</pre>
-  
-In this example there are three names:
-<dl>
-  <dt><i>myContext</i></dt>
-  <dd>The context name will typically identify either the application, or else a
-  module within an application or library.</dd>
-  
-  <dt><i>myRecord</i></dt>
-  <dd>The record name generally identifies some entity for which a set of
-  metrics are to be reported.  For example, you could have a record named 
-  "cacheStats" for reporting a number of statistics relating to the usage of
-  some cache in your application.</dd>
-  
-  <dt><i>myMetric</i></dt>
-  <dd>This identifies a particular metric.  For example, you might have metrics
-  named "cache_hits" and "cache_misses".
-  </dd>
-</dl>
-
-<h3>Tags</h3>
-
-In some cases it is useful to have multiple records with the same name. For 
-example, suppose that you want to report statistics about each disk on a computer. 
-In this case, the record name would be something like "diskStats", but you also
-need to identify the disk which is done by adding a <i>tag</i> to the record.
-The code could look something like this:
-<pre>
-    private MetricsRecord diskStats =
-            contextFactory.getContext("myContext").getRecord("diskStats");
-            
-    void reportDiskMetrics(String diskName, float diskBusy, float diskUsed) {
-        diskStats.setTag("diskName", diskName);
-        diskStats.setMetric("diskBusy", diskBusy);
-        diskStats.setMetric("diskUsed", diskUsed);
-        diskStats.update();
-    }
-</pre>
-
-<h3>Buffering and Callbacks</h3>
-
-Data is not sent immediately to the metrics system when 
-<code>MetricsRecord.update()</code> is called. Instead it is stored in an
-internal table, and the contents of the table are sent periodically.
-This can be important for two reasons:
-<ol>
-    <li>It means that a programmer is free to put calls to this API in an 
-    inner loop, since updates can be very frequent without slowing down
-    the application significantly.</li>
-    <li>Some implementations can gain efficiency by combining many metrics 
-    into a single UDP message.</li>
-</ol>
-
-The API provides a timer-based callback via the 
-<code>registerUpdater()</code> method.  The benefit of this
-versus using <code>java.util.Timer</code> is that the callbacks will be done 
-immediately before sending the data, making the data as current as possible.
-
-<h3>Configuration</h3>
-
-It is possible to programmatically examine and modify configuration data
-before creating a context, like this:
-<pre>
-    ContextFactory factory = ContextFactory.getFactory();
-    ... examine and/or modify factory attributes ...
-    MetricsContext context = factory.getContext("myContext");
-</pre>
-The factory attributes can be examined and modified using the following
-<code>ContextFactory</code>methods:
-<ul>
-    <li><code>Object getAttribute(String attributeName)</code></li>
-    <li><code>String[] getAttributeNames()</code></li>
-    <li><code>void setAttribute(String name, Object value)</code></li>
-    <li><code>void removeAttribute(attributeName)</code></li>
-</ul>
-
-<p/>
-<code>ContextFactory.getFactory()</code> initializes the factory attributes by
-reading the properties file <code>hadoop-metrics.properties</code> if it exists 
-on the class path.
-
-<p/>
-A factory attribute named:
-<pre>
-<i>contextName</i>.class
-</pre>
-should have as its value the fully qualified name of the class to be 
-instantiated by a call of the <code>CodeFactory</code> method
-<code>getContext(<i>contextName</i>)</code>.  If this factory attribute is not 
-specified, the default is to instantiate 
-<code>org.apache.hadoop.metrics.file.FileContext</code>.
-
-<p/>
-Other factory attributes are specific to a particular implementation of this 
-API and are documented elsewhere.  For example, configuration attributes for
-the file and Ganglia implementations can be found in the javadoc for 
-their respective packages.
-</body>
-</html>

+ 0 - 494
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java

@@ -1,494 +0,0 @@
-/*
- * AbstractMetricsContext.java
- *
- * 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.metrics.spi;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.TreeMap;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsException;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.Updater;
-
-/**
- * The main class of the Service Provider Interface.  This class should be
- * extended in order to integrate the Metrics API with a specific metrics
- * client library. <p/>
- *
- * This class implements the internal table of metric data, and the timer
- * on which data is to be sent to the metrics system.  Subclasses must
- * override the abstract <code>emitRecord</code> method in order to transmit
- * the data. <p/>
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public abstract class AbstractMetricsContext implements MetricsContext {
-    
-  private int period = MetricsContext.DEFAULT_PERIOD;
-  private Timer timer = null;
-    
-  private Set<Updater> updaters = new HashSet<Updater>(1);
-  private volatile boolean isMonitoring = false;
-    
-  private ContextFactory factory = null;
-  private String contextName = null;
-    
-  @InterfaceAudience.Private
-  public static class TagMap extends TreeMap<String,Object> {
-    private static final long serialVersionUID = 3546309335061952993L;
-    TagMap() {
-      super();
-    }
-    TagMap(TagMap orig) {
-      super(orig);
-    }
-    /**
-     * Returns true if this tagmap contains every tag in other.
-     */
-    public boolean containsAll(TagMap other) {
-      for (Map.Entry<String,Object> entry : other.entrySet()) {
-        Object value = get(entry.getKey());
-        if (value == null || !value.equals(entry.getValue())) {
-          // either key does not exist here, or the value is different
-          return false;
-        }
-      }
-      return true;
-    }
-  }
-  
-  @InterfaceAudience.Private
-  public static class MetricMap extends TreeMap<String,Number> {
-    private static final long serialVersionUID = -7495051861141631609L;
-    MetricMap() {
-      super();
-    }
-    MetricMap(MetricMap orig) {
-      super(orig);
-    }
-  }
-            
-  static class RecordMap extends HashMap<TagMap,MetricMap> {
-    private static final long serialVersionUID = 259835619700264611L;
-  }
-    
-  private Map<String,RecordMap> bufferedData = new HashMap<String,RecordMap>();
-    
-
-  /**
-   * Creates a new instance of AbstractMetricsContext
-   */
-  protected AbstractMetricsContext() {
-  }
-    
-  /**
-   * Initializes the context.
-   */
-  @Override
-  public void init(String contextName, ContextFactory factory) 
-  {
-    this.contextName = contextName;
-    this.factory = factory;
-  }
-    
-  /**
-   * Convenience method for subclasses to access factory attributes.
-   */
-  protected String getAttribute(String attributeName) {
-    String factoryAttribute = contextName + "." + attributeName;
-    return (String) factory.getAttribute(factoryAttribute);  
-  }
-    
-  /**
-   * Returns an attribute-value map derived from the factory attributes
-   * by finding all factory attributes that begin with 
-   * <i>contextName</i>.<i>tableName</i>.  The returned map consists of
-   * those attributes with the contextName and tableName stripped off.
-   */
-  protected Map<String,String> getAttributeTable(String tableName) {
-    String prefix = contextName + "." + tableName + ".";
-    Map<String,String> result = new HashMap<String,String>();
-    for (String attributeName : factory.getAttributeNames()) {
-      if (attributeName.startsWith(prefix)) {
-        String name = attributeName.substring(prefix.length());
-        String value = (String) factory.getAttribute(attributeName);
-        result.put(name, value);
-      }
-    }
-    return result;
-  }
-    
-  /**
-   * Returns the context name.
-   */
-  @Override
-  public String getContextName() {
-    return contextName;
-  }
-    
-  /**
-   * Returns the factory by which this context was created.
-   */
-  public ContextFactory getContextFactory() {
-    return factory;
-  }
-    
-  /**
-   * Starts or restarts monitoring, the emitting of metrics records.
-   */
-  @Override
-  public synchronized void startMonitoring()
-    throws IOException {
-    if (!isMonitoring) {
-      startTimer();
-      isMonitoring = true;
-    }
-  }
-    
-  /**
-   * Stops monitoring.  This does not free buffered data. 
-   * @see #close()
-   */
-  @Override
-  public synchronized void stopMonitoring() {
-    if (isMonitoring) {
-      stopTimer();
-      isMonitoring = false;
-    }
-  }
-    
-  /**
-   * Returns true if monitoring is currently in progress.
-   */
-  @Override
-  public boolean isMonitoring() {
-    return isMonitoring;
-  }
-    
-  /**
-   * Stops monitoring and frees buffered data, returning this
-   * object to its initial state.  
-   */
-  @Override
-  public synchronized void close() {
-    stopMonitoring();
-    clearUpdaters();
-  } 
-    
-  /**
-   * Creates a new AbstractMetricsRecord instance with the given <code>recordName</code>.
-   * Throws an exception if the metrics implementation is configured with a fixed
-   * set of record names and <code>recordName</code> is not in that set.
-   * 
-   * @param recordName the name of the record
-   * @throws MetricsException if recordName conflicts with configuration data
-   */
-  @Override
-  public final synchronized MetricsRecord createRecord(String recordName) {
-    if (bufferedData.get(recordName) == null) {
-      bufferedData.put(recordName, new RecordMap());
-    }
-    return newRecord(recordName);
-  }
-    
-  /**
-   * Subclasses should override this if they subclass MetricsRecordImpl.
-   * @param recordName the name of the record
-   * @return newly created instance of MetricsRecordImpl or subclass
-   */
-  protected MetricsRecord newRecord(String recordName) {
-    return new MetricsRecordImpl(recordName, this);
-  }
-    
-  /**
-   * Registers a callback to be called at time intervals determined by
-   * the configuration.
-   *
-   * @param updater object to be run periodically; it should update
-   * some metrics records 
-   */
-  @Override
-  public synchronized void registerUpdater(final Updater updater) {
-    if (!updaters.contains(updater)) {
-      updaters.add(updater);
-    }
-  }
-    
-  /**
-   * Removes a callback, if it exists.
-   *
-   * @param updater object to be removed from the callback list
-   */
-  @Override
-  public synchronized void unregisterUpdater(Updater updater) {
-    updaters.remove(updater);
-  }
-    
-  private synchronized void clearUpdaters() {
-    updaters.clear();
-  }
-    
-  /**
-   * Starts timer if it is not already started
-   */
-  private synchronized void startTimer() {
-    if (timer == null) {
-      timer = new Timer("Timer thread for monitoring " + getContextName(), 
-                        true);
-      TimerTask task = new TimerTask() {
-          @Override
-          public void run() {
-            try {
-              timerEvent();
-            } catch (IOException ioe) {
-              ioe.printStackTrace();
-            }
-          }
-        };
-      long millis = period * 1000;
-      timer.scheduleAtFixedRate(task, millis, millis);
-    }
-  }
-    
-  /**
-   * Stops timer if it is running
-   */
-  private synchronized void stopTimer() {
-    if (timer != null) {
-      timer.cancel();
-      timer = null;
-    }
-  }
-    
-  /**
-   * Timer callback.
-   */
-  private void timerEvent() throws IOException {
-    if (isMonitoring) {
-      Collection<Updater> myUpdaters;
-      synchronized (this) {
-        myUpdaters = new ArrayList<Updater>(updaters);
-      }     
-      // Run all the registered updates without holding a lock
-      // on this context
-      for (Updater updater : myUpdaters) {
-        try {
-          updater.doUpdates(this);
-        } catch (Throwable throwable) {
-          throwable.printStackTrace();
-        }
-      }
-      emitRecords();
-    }
-  }
-    
-  /**
-   *  Emits the records.
-   */
-  private synchronized void emitRecords() throws IOException {
-    for (Map.Entry<String,RecordMap> recordEntry : bufferedData.entrySet()) {
-      RecordMap recordMap = recordEntry.getValue();
-      synchronized (recordMap) {
-        Set<Entry<TagMap, MetricMap>> entrySet = recordMap.entrySet ();
-        for (Entry<TagMap, MetricMap> entry : entrySet) {
-          OutputRecord outRec = new OutputRecord(entry.getKey(), entry.getValue());
-          emitRecord(contextName, recordEntry.getKey(), outRec);
-        }
-      }
-    }
-    flush();
-  }
-  
-  /**
-   * Retrieves all the records managed by this MetricsContext.
-   * Useful for monitoring systems that are polling-based.
-   * @return A non-null collection of all monitoring records.
-   */
-  @Override
-  public synchronized Map<String, Collection<OutputRecord>> getAllRecords() {
-    Map<String, Collection<OutputRecord>> out = new TreeMap<String, Collection<OutputRecord>>();
-    for (Map.Entry<String,RecordMap> recordEntry : bufferedData.entrySet()) {
-      RecordMap recordMap = recordEntry.getValue();
-      synchronized (recordMap) {
-        List<OutputRecord> records = new ArrayList<OutputRecord>();
-        Set<Entry<TagMap, MetricMap>> entrySet = recordMap.entrySet();
-        for (Entry<TagMap, MetricMap> entry : entrySet) {
-          OutputRecord outRec = new OutputRecord(entry.getKey(), entry.getValue());
-          records.add(outRec);
-        }
-        out.put(recordEntry.getKey(), records);
-      }
-    }
-    return out;
-  }
-
-  /**
-   * Sends a record to the metrics system.
-   */
-  protected abstract void emitRecord(String contextName, String recordName, 
-                                     OutputRecord outRec) throws IOException;
-    
-  /**
-   * Called each period after all records have been emitted, this method does nothing.
-   * Subclasses may override it in order to perform some kind of flush.
-   */
-  protected void flush() throws IOException {
-  }
-    
-  /**
-   * Called by MetricsRecordImpl.update().  Creates or updates a row in
-   * the internal table of metric data.
-   */
-  protected void update(MetricsRecordImpl record) {
-    String recordName = record.getRecordName();
-    TagMap tagTable = record.getTagTable();
-    Map<String,MetricValue> metricUpdates = record.getMetricTable();
-        
-    RecordMap recordMap = getRecordMap(recordName);
-    synchronized (recordMap) {
-      MetricMap metricMap = recordMap.get(tagTable);
-      if (metricMap == null) {
-        metricMap = new MetricMap();
-        TagMap tagMap = new TagMap(tagTable); // clone tags
-        recordMap.put(tagMap, metricMap);
-      }
-
-      Set<Entry<String, MetricValue>> entrySet = metricUpdates.entrySet();
-      for (Entry<String, MetricValue> entry : entrySet) {
-        String metricName = entry.getKey ();
-        MetricValue updateValue = entry.getValue ();
-        Number updateNumber = updateValue.getNumber();
-        Number currentNumber = metricMap.get(metricName);
-        if (currentNumber == null || updateValue.isAbsolute()) {
-          metricMap.put(metricName, updateNumber);
-        }
-        else {
-          Number newNumber = sum(updateNumber, currentNumber);
-          metricMap.put(metricName, newNumber);
-        }
-      }
-    }
-  }
-    
-  private synchronized RecordMap getRecordMap(String recordName) {
-    return bufferedData.get(recordName);
-  }
-    
-  /**
-   * Adds two numbers, coercing the second to the type of the first.
-   *
-   */
-  private Number sum(Number a, Number b) {
-    if (a instanceof Integer) {
-      return Integer.valueOf(a.intValue() + b.intValue());
-    }
-    else if (a instanceof Float) {
-      return new Float(a.floatValue() + b.floatValue());
-    }
-    else if (a instanceof Short) {
-      return Short.valueOf((short)(a.shortValue() + b.shortValue()));
-    }
-    else if (a instanceof Byte) {
-      return Byte.valueOf((byte)(a.byteValue() + b.byteValue()));
-    }
-    else if (a instanceof Long) {
-      return Long.valueOf((a.longValue() + b.longValue()));
-    }
-    else {
-      // should never happen
-      throw new MetricsException("Invalid number type");
-    }
-            
-  }
-    
-  /**
-   * Called by MetricsRecordImpl.remove().  Removes all matching rows in
-   * the internal table of metric data.  A row matches if it has the same
-   * tag names and values as record, but it may also have additional
-   * tags.
-   */    
-  protected void remove(MetricsRecordImpl record) {
-    String recordName = record.getRecordName();
-    TagMap tagTable = record.getTagTable();
-        
-    RecordMap recordMap = getRecordMap(recordName);
-    synchronized (recordMap) {
-      Iterator<TagMap> it = recordMap.keySet().iterator();
-      while (it.hasNext()) {
-        TagMap rowTags = it.next();
-        if (rowTags.containsAll(tagTable)) {
-          it.remove();
-        }
-      }
-    }
-  }
-    
-  /**
-   * Returns the timer period.
-   */
-  @Override
-  public int getPeriod() {
-    return period;
-  }
-    
-  /**
-   * Sets the timer period
-   */
-  protected void setPeriod(int period) {
-    this.period = period;
-  }
-  
-  /**
-   * If a period is set in the attribute passed in, override
-   * the default with it.
-   */
-  protected void parseAndSetPeriod(String attributeName) {
-    String periodStr = getAttribute(attributeName);
-    if (periodStr != null) {
-      int period = 0;
-      try {
-        period = Integer.parseInt(periodStr);
-      } catch (NumberFormatException nfe) {
-      }
-      if (period <= 0) {
-        throw new MetricsException("Invalid period: " + periodStr);
-      }
-      setPeriod(period);
-    }
-  }
-}

+ 0 - 206
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/CompositeContext.java

@@ -1,206 +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.metrics.spi;
-
-import java.io.IOException;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.util.ArrayList;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-
-/**
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CompositeContext extends AbstractMetricsContext {
-
-  private static final Log LOG = LogFactory.getLog(CompositeContext.class);
-  private static final String ARITY_LABEL = "arity";
-  private static final String SUB_FMT = "%s.sub%d";
-  private final ArrayList<MetricsContext> subctxt =
-    new ArrayList<MetricsContext>();
-
-  @InterfaceAudience.Private
-  public CompositeContext() {
-  }
-
-  @Override
-  @InterfaceAudience.Private
-  public void init(String contextName, ContextFactory factory) {
-    super.init(contextName, factory);
-    int nKids;
-    try {
-      String sKids = getAttribute(ARITY_LABEL);
-      nKids = Integer.parseInt(sKids);
-    } catch (Exception e) {
-      LOG.error("Unable to initialize composite metric " + contextName +
-                ": could not init arity", e);
-      return;
-    }
-    for (int i = 0; i < nKids; ++i) {
-      MetricsContext ctxt = MetricsUtil.getContext(
-          String.format(SUB_FMT, contextName, i), contextName);
-      if (null != ctxt) {
-        subctxt.add(ctxt);
-      }
-    }
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public MetricsRecord newRecord(String recordName) {
-    return (MetricsRecord) Proxy.newProxyInstance(
-        MetricsRecord.class.getClassLoader(),
-        new Class[] { MetricsRecord.class },
-        new MetricsRecordDelegator(recordName, subctxt));
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  protected void emitRecord(String contextName, String recordName,
-      OutputRecord outRec) throws IOException {
-    for (MetricsContext ctxt : subctxt) {
-      try {
-        ((AbstractMetricsContext)ctxt).emitRecord(
-          contextName, recordName, outRec);
-        if (contextName == null || recordName == null || outRec == null) {
-          throw new IOException(contextName + ":" + recordName + ":" + outRec);
-        }
-      } catch (IOException e) {
-        LOG.warn("emitRecord failed: " + ctxt.getContextName(), e);
-      }
-    }
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  protected void flush() throws IOException {
-    for (MetricsContext ctxt : subctxt) {
-      try {
-        ((AbstractMetricsContext)ctxt).flush();
-      } catch (IOException e) {
-        LOG.warn("flush failed: " + ctxt.getContextName(), e);
-      }
-    }
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public void startMonitoring() throws IOException {
-    for (MetricsContext ctxt : subctxt) {
-      try {
-        ctxt.startMonitoring();
-      } catch (IOException e) {
-        LOG.warn("startMonitoring failed: " + ctxt.getContextName(), e);
-      }
-    }
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public void stopMonitoring() {
-    for (MetricsContext ctxt : subctxt) {
-      ctxt.stopMonitoring();
-    }
-  }
-
-  /**
-   * Return true if all subcontexts are monitoring.
-   */
-  @InterfaceAudience.Private
-  @Override
-  public boolean isMonitoring() {
-    boolean ret = true;
-    for (MetricsContext ctxt : subctxt) {
-      ret &= ctxt.isMonitoring();
-    }
-    return ret;
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public void close() {
-    for (MetricsContext ctxt : subctxt) {
-      ctxt.close();
-    }
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public void registerUpdater(Updater updater) {
-    for (MetricsContext ctxt : subctxt) {
-      ctxt.registerUpdater(updater);
-    }
-  }
-
-  @InterfaceAudience.Private
-  @Override
-  public void unregisterUpdater(Updater updater) {
-    for (MetricsContext ctxt : subctxt) {
-      ctxt.unregisterUpdater(updater);
-    }
-  }
-
-  private static class MetricsRecordDelegator implements InvocationHandler {
-    private static final Method m_getRecordName = initMethod();
-    private static Method initMethod() {
-      try {
-        return MetricsRecord.class.getMethod("getRecordName", new Class[0]);
-      } catch (Exception e) {
-        throw new RuntimeException("Internal error", e);
-      }
-    }
-
-    private final String recordName;
-    private final ArrayList<MetricsRecord> subrecs;
-
-    MetricsRecordDelegator(String recordName, ArrayList<MetricsContext> ctxts) {
-      this.recordName = recordName;
-      this.subrecs = new ArrayList<MetricsRecord>(ctxts.size());
-      for (MetricsContext ctxt : ctxts) {
-        subrecs.add(ctxt.createRecord(recordName));
-      }
-    }
-
-    @Override
-    public Object invoke(Object p, Method m, Object[] args) throws Throwable {
-      if (m_getRecordName.equals(m)) {
-        return recordName;
-      }
-      assert Void.TYPE.equals(m.getReturnType());
-      for (MetricsRecord rec : subrecs) {
-        m.invoke(rec, args);
-      }
-      return null;
-    }
-  }
-
-}

+ 0 - 58
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/MetricValue.java

@@ -1,58 +0,0 @@
-/*
- * MetricValue.java
- *
- * 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.metrics.spi;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * A Number that is either an absolute or an incremental amount.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class MetricValue {
-    
-  public static final boolean ABSOLUTE = false;
-  public static final boolean INCREMENT = true;
-    
-  private boolean isIncrement;
-  private Number number;
-    
-  /** Creates a new instance of MetricValue */
-  public MetricValue(Number number, boolean isIncrement) {
-    this.number = number;
-    this.isIncrement = isIncrement;
-  }
-
-  public boolean isIncrement() {
-    return isIncrement;
-  }
-    
-  public boolean isAbsolute() {
-    return !isIncrement;
-  }
-
-  public Number getNumber() {
-    return number;
-  }
-    
-}

+ 0 - 304
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/MetricsRecordImpl.java

@@ -1,304 +0,0 @@
-/*
- * MetricsRecordImpl.java
- *
- * 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.metrics.spi;
-
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.MetricsException;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
-
-/**
- * An implementation of MetricsRecord.  Keeps a back-pointer to the context
- * from which it was created, and delegates back to it on <code>update</code>
- * and <code>remove()</code>.
- *
- * @deprecated Use {@link org.apache.hadoop.metrics2.impl.MetricsRecordImpl}
- * instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class MetricsRecordImpl implements MetricsRecord {
-    
-  private TagMap tagTable = new TagMap();
-  private Map<String,MetricValue> metricTable = new LinkedHashMap<String,MetricValue>();
-    
-  private String recordName;
-  private AbstractMetricsContext context;
-    
-    
-  /** Creates a new instance of FileRecord */
-  protected MetricsRecordImpl(String recordName, AbstractMetricsContext context)
-  {
-    this.recordName = recordName;
-    this.context = context;
-  }
-    
-  /**
-   * Returns the record name. 
-   *
-   * @return the record name
-   */
-  @Override
-  public String getRecordName() {
-    return recordName;
-  }
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  @Override
-  public void setTag(String tagName, String tagValue) {
-    if (tagValue == null) {
-      tagValue = "";
-    }
-    tagTable.put(tagName, tagValue);
-  }
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  @Override
-  public void setTag(String tagName, int tagValue) {
-    tagTable.put(tagName, Integer.valueOf(tagValue));
-  }
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  @Override
-  public void setTag(String tagName, long tagValue) {
-    tagTable.put(tagName, Long.valueOf(tagValue));
-  }
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  @Override
-  public void setTag(String tagName, short tagValue) {
-    tagTable.put(tagName, Short.valueOf(tagValue));
-  }
-    
-  /**
-   * Sets the named tag to the specified value.
-   *
-   * @param tagName name of the tag
-   * @param tagValue new value of the tag
-   * @throws MetricsException if the tagName conflicts with the configuration
-   */
-  @Override
-  public void setTag(String tagName, byte tagValue) {
-    tagTable.put(tagName, Byte.valueOf(tagValue));
-  }
-    
-  /**
-   * Removes any tag of the specified name.
-   */
-  @Override
-  public void removeTag(String tagName) {
-    tagTable.remove(tagName);
-  }
-  
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void setMetric(String metricName, int metricValue) {
-    setAbsolute(metricName, Integer.valueOf(metricValue));
-  }
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void setMetric(String metricName, long metricValue) {
-    setAbsolute(metricName, Long.valueOf(metricValue));
-  }
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void setMetric(String metricName, short metricValue) {
-    setAbsolute(metricName, Short.valueOf(metricValue));
-  }
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void setMetric(String metricName, byte metricValue) {
-    setAbsolute(metricName, Byte.valueOf(metricValue));
-  }
-    
-  /**
-   * Sets the named metric to the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue new value of the metric
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void setMetric(String metricName, float metricValue) {
-    setAbsolute(metricName, new Float(metricValue));
-  }
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void incrMetric(String metricName, int metricValue) {
-    setIncrement(metricName, Integer.valueOf(metricValue));
-  }
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void incrMetric(String metricName, long metricValue) {
-    setIncrement(metricName, Long.valueOf(metricValue));
-  }
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void incrMetric(String metricName, short metricValue) {
-    setIncrement(metricName, Short.valueOf(metricValue));
-  }
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void incrMetric(String metricName, byte metricValue) {
-    setIncrement(metricName, Byte.valueOf(metricValue));
-  }
-    
-  /**
-   * Increments the named metric by the specified value.
-   *
-   * @param metricName name of the metric
-   * @param metricValue incremental value
-   * @throws MetricsException if the metricName or the type of the metricValue 
-   * conflicts with the configuration
-   */
-  @Override
-  public void incrMetric(String metricName, float metricValue) {
-    setIncrement(metricName, new Float(metricValue));
-  }
-    
-  private void setAbsolute(String metricName, Number metricValue) {
-    metricTable.put(metricName, new MetricValue(metricValue, MetricValue.ABSOLUTE));
-  }
-    
-  private void setIncrement(String metricName, Number metricValue) {
-    metricTable.put(metricName, new MetricValue(metricValue, MetricValue.INCREMENT));
-  }
-    
-  /**
-   * Updates the table of buffered data which is to be sent periodically.
-   * If the tag values match an existing row, that row is updated; 
-   * otherwise, a new row is added.
-   */
-  @Override
-  public void update() {
-    context.update(this);
-  }
-    
-  /**
-   * Removes the row, if it exists, in the buffered data table having tags 
-   * that equal the tags that have been set on this record. 
-   */
-  @Override
-  public void remove() {
-    context.remove(this);
-  }
-
-  TagMap getTagTable() {
-    return tagTable;
-  }
-
-  Map<String, MetricValue> getMetricTable() {
-    return metricTable;
-  }
-}

+ 0 - 61
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/NoEmitMetricsContext.java

@@ -1,61 +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.metrics.spi;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.MetricsServlet;
-
-/** 
- * A MetricsContext that does not emit data, but, unlike NullContextWithUpdate,
- * does save it for retrieval with getAllRecords().
- * 
- * This is useful if you want to support {@link MetricsServlet}, but
- * not emit metrics in any other way.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class NoEmitMetricsContext extends AbstractMetricsContext {
-    
-    private static final String PERIOD_PROPERTY = "period";
-      
-    /** Creates a new instance of NullContextWithUpdateThread */
-    @InterfaceAudience.Private
-    public NoEmitMetricsContext() {
-    }
-    
-    @Override
-    @InterfaceAudience.Private
-    public void init(String contextName, ContextFactory factory) {
-      super.init(contextName, factory);
-      parseAndSetPeriod(PERIOD_PROPERTY);
-    }
-     
-    /**
-     * Do-nothing version of emitRecord
-     */
-    @Override
-    @InterfaceAudience.Private
-    protected void emitRecord(String contextName, String recordName,
-                              OutputRecord outRec) {
-    }
-}

+ 0 - 74
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/NullContext.java

@@ -1,74 +0,0 @@
-/*
- * NullContext.java
- * 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.metrics.spi;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Null metrics context: a metrics context which does nothing.  Used as the
- * default context, so that no performance data is emitted if no configuration
- * data is found.
- * 
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class NullContext extends AbstractMetricsContext {
-    
-  /** Creates a new instance of NullContext */
-  @InterfaceAudience.Private
-  public NullContext() {
-  }
-    
-  /**
-   * Do-nothing version of startMonitoring
-   */
-  @Override
-  @InterfaceAudience.Private
-  public void startMonitoring() {
-  }
-    
-  /**
-   * Do-nothing version of emitRecord
-   */
-  @Override
-  @InterfaceAudience.Private
-  protected void emitRecord(String contextName, String recordName,
-                            OutputRecord outRec) 
-  {}
-    
-  /**
-   * Do-nothing version of update
-   */
-  @Override
-  @InterfaceAudience.Private
-  protected void update(MetricsRecordImpl record) {
-  }
-    
-  /**
-   * Do-nothing version of remove
-   */
-  @Override
-  @InterfaceAudience.Private
-  protected void remove(MetricsRecordImpl record) {
-  }
-}

+ 0 - 82
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/NullContextWithUpdateThread.java

@@ -1,82 +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.metrics.spi;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.ContextFactory;
-
-/**
- * A null context which has a thread calling 
- * periodically when monitoring is started. This keeps the data sampled 
- * correctly.
- * In all other respects, this is like the NULL context: No data is emitted.
- * This is suitable for Monitoring systems like JMX which reads the metrics
- *  when someone reads the data from JMX.
- * 
- * The default impl of start and stop monitoring:
- *  is the AbstractMetricsContext is good enough.
- * 
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class NullContextWithUpdateThread extends AbstractMetricsContext {
-  
-  private static final String PERIOD_PROPERTY = "period";
-    
-  /** Creates a new instance of NullContextWithUpdateThread */
-  @InterfaceAudience.Private
-  public NullContextWithUpdateThread() {
-  }
-  
-  @Override
-  @InterfaceAudience.Private
-  public void init(String contextName, ContextFactory factory) {
-    super.init(contextName, factory);
-    parseAndSetPeriod(PERIOD_PROPERTY);
-  }
-   
-    
-  /**
-   * Do-nothing version of emitRecord
-   */
-  @Override
-  @InterfaceAudience.Private
-  protected void emitRecord(String contextName, String recordName,
-                            OutputRecord outRec) 
-  {}
-    
-  /**
-   * Do-nothing version of update
-   */
-  @Override
-  @InterfaceAudience.Private
-  protected void update(MetricsRecordImpl record) {
-  }
-    
-  /**
-   * Do-nothing version of remove
-   */
-  @Override
-  @InterfaceAudience.Private
-  protected void remove(MetricsRecordImpl record) {
-  }
-}

+ 0 - 93
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/OutputRecord.java

@@ -1,93 +0,0 @@
-/*
- * OutputRecord.java
- *
- * 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.metrics.spi;
-
-import java.util.Collections;
-import java.util.Set;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
-
-/**
- * Represents a record of metric data to be sent to a metrics system.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class OutputRecord {
-    
-  private TagMap tagMap;
-  private MetricMap metricMap;
-    
-  /** Creates a new instance of OutputRecord */
-  OutputRecord(TagMap tagMap, MetricMap metricMap) {
-    this.tagMap = tagMap;
-    this.metricMap = metricMap;
-  }
-    
-  /**
-   * Returns the set of tag names
-   */
-  public Set<String> getTagNames() {
-    return Collections.unmodifiableSet(tagMap.keySet());
-  }
-    
-  /**
-   * Returns a tag object which is can be a String, Integer, Short or Byte.
-   *
-   * @return the tag value, or null if there is no such tag
-   */
-  public Object getTag(String name) {
-    return tagMap.get(name);
-  }
-    
-  /**
-   * Returns the set of metric names.
-   */
-  public Set<String> getMetricNames() {
-    return Collections.unmodifiableSet(metricMap.keySet());
-  }
-    
-  /**
-   * Returns the metric object which can be a Float, Integer, Short or Byte.
-   */
-  public Number getMetric(String name) {
-    return metricMap.get(name);
-  }
-  
-
-  /**
-   * Returns a copy of this record's tags.
-   */
-  public TagMap getTagsCopy() {
-    return new TagMap(tagMap);
-  }
-  
-  /**
-   * Returns a copy of this record's metrics.
-   */
-  public MetricMap getMetricsCopy() {
-    return new MetricMap(metricMap);
-  }
-}

+ 0 - 68
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/Util.java

@@ -1,68 +0,0 @@
-/*
- * Util.java
- *
- * 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.metrics.spi;
-
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.net.NetUtils;
-
-/**
- * Static utility methods
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class Util {
-    
-  /**
-   * This class is not intended to be instantiated
-   */
-  private Util() {}
-    
-  /**
-   * Parses a space and/or comma separated sequence of server specifications
-   * of the form <i>hostname</i> or <i>hostname:port</i>.  If 
-   * the specs string is null, defaults to localhost:defaultPort.
-   * 
-   * @return a list of InetSocketAddress objects.
-   */
-  public static List<InetSocketAddress> parse(String specs, int defaultPort) {
-    List<InetSocketAddress> result = new ArrayList<InetSocketAddress>(1);
-    if (specs == null) {
-      result.add(new InetSocketAddress("localhost", defaultPort));
-    }
-    else {
-      String[] specStrings = specs.split("[ ,]+");
-      for (String specString : specStrings) {
-        result.add(NetUtils.createSocketAddr(specString, defaultPort));
-      }
-    }
-    return result;
-  }
-    
-}

+ 0 - 36
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/spi/package.html

@@ -1,36 +0,0 @@
-<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN">
-<html>
-
-<!--
-   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.
--->
-
-  <head>
-    <title>org.apache.hadoop.metrics.spi</title>
-  </head>
-  <body>
-The Service Provider Interface for the Metrics API.  This package provides
-an interface allowing a variety of metrics reporting implementations to be
-plugged in to the Metrics API.  Examples of such implementations can be found 
-in the packages <code>org.apache.hadoop.metrics.file</code> and
-<code>org.apache.hadoop.metrics.ganglia</code>.<p/>
-
-Plugging in an implementation involves writing a concrete subclass of 
-<code>AbstractMetricsContext</code>.  The subclass should get its
- configuration information using the <code>getAttribute(<i>attributeName</i>)</code>
- method.
-  </body>
-</html>

+ 0 - 92
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MBeanUtil.java

@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics.util;
-
-import java.lang.management.ManagementFactory;
-
-import javax.management.InstanceNotFoundException;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.InstanceAlreadyExistsException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-
-/**
- * This util class provides a method to register an MBean using
- * our standard naming convention as described in the doc
- *  for {link {@link #registerMBean(String, String, Object)}
- *
- * @deprecated Use {@link org.apache.hadoop.metrics2.util.MBeans} instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public class MBeanUtil {
-	
-  /**
-   * Register the MBean using our standard MBeanName format
-   * "hadoop:service=<serviceName>,name=<nameName>"
-   * Where the <serviceName> and <nameName> are the supplied parameters
-   *    
-   * @param serviceName
-   * @param nameName
-   * @param theMbean - the MBean to register
-   * @return the named used to register the MBean
-   */	
-  static public ObjectName registerMBean(final String serviceName, 
-		  							final String nameName,
-		  							final Object theMbean) {
-    final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-    ObjectName name = getMBeanName(serviceName, nameName);
-    try {
-      mbs.registerMBean(theMbean, name);
-      return name;
-    } catch (InstanceAlreadyExistsException ie) {
-      // Ignore if instance already exists 
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-    return null;
-  }
-  
-  static public void unregisterMBean(ObjectName mbeanName) {
-    final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-    if (mbeanName == null) 
-        return;
-    try {
-      mbs.unregisterMBean(mbeanName);
-    } catch (InstanceNotFoundException e ) {
-      // ignore
-    } catch (Exception e) {
-      e.printStackTrace();
-    } 
-  }
-  
-  static private ObjectName getMBeanName(final String serviceName,
-		  								 final String nameName) {
-    ObjectName name = null;
-    try {
-      name = new ObjectName("hadoop:" +
-                  "service=" + serviceName + ",name=" + nameName);
-    } catch (MalformedObjectNameException e) {
-      e.printStackTrace();
-    }
-    return name;
-  }
-}

+ 0 - 51
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsBase.java

@@ -1,51 +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.metrics.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.metrics.MetricsRecord;
-
-/**
- * 
- * This is base class for all metrics
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.Private
-public abstract class MetricsBase {
-  public static final String NO_DESCRIPTION = "NoDescription";
-  final private String name;
-  final private String description;
-  
-  protected MetricsBase(final String nam) {
-    name = nam;
-    description = NO_DESCRIPTION;
-  }
-  
-  protected MetricsBase(final String nam, final String desc) {
-    name = nam;
-    description = desc;
-  }
-  
-  public abstract void pushMetric(final MetricsRecord mr);
-  
-  public String getName() { return name; }
-  public String getDescription() { return description; };
-
-}

+ 0 - 229
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsDynamicMBeanBase.java

@@ -1,229 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics.util;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.management.Attribute;
-import javax.management.AttributeList;
-import javax.management.AttributeNotFoundException;
-import javax.management.DynamicMBean;
-import javax.management.InvalidAttributeValueException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanException;
-import javax.management.MBeanInfo;
-import javax.management.MBeanOperationInfo;
-import javax.management.ReflectionException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.metrics.MetricsUtil;
-
-
-
-/**
- * This abstract base class facilitates creating dynamic mbeans automatically from
- * metrics. 
- * The metrics constructors registers metrics in a registry. 
- * Different categories of metrics should be in differnt classes with their own
- * registry (as in NameNodeMetrics and DataNodeMetrics).
- * Then the MBean can be created passing the registry to the constructor.
- * The MBean should be then registered using a mbean name (example):
- *  MetricsHolder myMetrics = new MetricsHolder(); // has metrics and registry
- *  MetricsTestMBean theMBean = new MetricsTestMBean(myMetrics.mregistry);
- *  ObjectName mbeanName = MBeanUtil.registerMBean("ServiceFoo",
- *                "TestStatistics", theMBean);
- * 
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public abstract class MetricsDynamicMBeanBase implements DynamicMBean {
-  private final static String AVG_TIME = "AvgTime";
-  private final static String MIN_TIME = "MinTime";
-  private final static String MAX_TIME = "MaxTime";
-  private final static String NUM_OPS = "NumOps";
-  private final static String RESET_ALL_MIN_MAX_OP = "resetAllMinMax";
-  private MetricsRegistry metricsRegistry;
-  private MBeanInfo mbeanInfo;
-  private Map<String, MetricsBase> metricsRateAttributeMod;
-  private int numEntriesInRegistry = 0;
-  private String mbeanDescription;
-  
-  protected MetricsDynamicMBeanBase(final MetricsRegistry mr, final String aMBeanDescription) {
-    metricsRegistry = mr;
-    mbeanDescription = aMBeanDescription;
-    metricsRateAttributeMod = new ConcurrentHashMap<String, MetricsBase>();
-    createMBeanInfo();
-  }
-  
-  private void updateMbeanInfoIfMetricsListChanged()  {
-    if (numEntriesInRegistry != metricsRegistry.size())
-      createMBeanInfo();
-  }
-  
-  private void createMBeanInfo() {
-    boolean needsMinMaxResetOperation = false;
-    List<MBeanAttributeInfo> attributesInfo = new ArrayList<MBeanAttributeInfo>();
-    MBeanOperationInfo[] operationsInfo = null;
-    numEntriesInRegistry = metricsRegistry.size();
-    
-    for (MetricsBase o : metricsRegistry.getMetricsList()) {
-
-      if (MetricsTimeVaryingRate.class.isInstance(o)) {
-        // For each of the metrics there are 3 different attributes
-        attributesInfo.add(new MBeanAttributeInfo(o.getName() + NUM_OPS, "java.lang.Integer",
-            o.getDescription(), true, false, false));
-        attributesInfo.add(new MBeanAttributeInfo(o.getName() + AVG_TIME, "java.lang.Long",
-            o.getDescription(), true, false, false));
-        attributesInfo.add(new MBeanAttributeInfo(o.getName() + MIN_TIME, "java.lang.Long",
-            o.getDescription(), true, false, false));
-        attributesInfo.add(new MBeanAttributeInfo(o.getName() + MAX_TIME, "java.lang.Long",
-            o.getDescription(), true, false, false));
-        needsMinMaxResetOperation = true;  // the min and max can be reset.
-        
-        // Note the special attributes (AVG_TIME, MIN_TIME, ..) are derived from metrics 
-        // Rather than check for the suffix we store them in a map.
-        metricsRateAttributeMod.put(o.getName() + NUM_OPS, o);
-        metricsRateAttributeMod.put(o.getName() + AVG_TIME, o);
-        metricsRateAttributeMod.put(o.getName() + MIN_TIME, o);
-        metricsRateAttributeMod.put(o.getName() + MAX_TIME, o);
-        
-      }  else if ( MetricsIntValue.class.isInstance(o) || MetricsTimeVaryingInt.class.isInstance(o) ) {
-        attributesInfo.add(new MBeanAttributeInfo(o.getName(), "java.lang.Integer",
-            o.getDescription(), true, false, false)); 
-      } else if ( MetricsLongValue.class.isInstance(o) || MetricsTimeVaryingLong.class.isInstance(o) ) {
-        attributesInfo.add(new MBeanAttributeInfo(o.getName(), "java.lang.Long",
-            o.getDescription(), true, false, false));     
-      } else {
-        MetricsUtil.LOG.error("unknown metrics type: " + o.getClass().getName());
-      }
-
-      if (needsMinMaxResetOperation) {
-        operationsInfo = new MBeanOperationInfo[] {
-            new MBeanOperationInfo(RESET_ALL_MIN_MAX_OP, "Reset (zero) All Min Max",
-                    null, "void", MBeanOperationInfo.ACTION) };
-      }
-    }
-    MBeanAttributeInfo[] attrArray = new MBeanAttributeInfo[attributesInfo.size()];
-    mbeanInfo =  new MBeanInfo(this.getClass().getName(), mbeanDescription, 
-        attributesInfo.toArray(attrArray), null, operationsInfo, null);
-  }
-  
-  @Override
-  public Object getAttribute(String attributeName) throws AttributeNotFoundException,
-      MBeanException, ReflectionException {
-    if (attributeName == null || attributeName.isEmpty()) 
-      throw new IllegalArgumentException();
-    
-    updateMbeanInfoIfMetricsListChanged();
-    
-    Object o = metricsRateAttributeMod.get(attributeName);
-    if (o == null) {
-      o = metricsRegistry.get(attributeName);
-    }
-    if (o == null)
-      throw new AttributeNotFoundException();
-    
-    if (o instanceof MetricsIntValue)
-      return ((MetricsIntValue) o).get();
-    else if (o instanceof MetricsLongValue)
-      return ((MetricsLongValue) o).get();
-    else if (o instanceof MetricsTimeVaryingInt)
-      return ((MetricsTimeVaryingInt) o).getPreviousIntervalValue();
-    else if (o instanceof MetricsTimeVaryingLong)
-      return ((MetricsTimeVaryingLong) o).getPreviousIntervalValue();
-    else if (o instanceof MetricsTimeVaryingRate) {
-      MetricsTimeVaryingRate or = (MetricsTimeVaryingRate) o;
-      if (attributeName.endsWith(NUM_OPS))
-        return or.getPreviousIntervalNumOps();
-      else if (attributeName.endsWith(AVG_TIME))
-        return or.getPreviousIntervalAverageTime();
-      else if (attributeName.endsWith(MIN_TIME))
-        return or.getMinTime();
-      else if (attributeName.endsWith(MAX_TIME))
-        return or.getMaxTime();
-      else {
-        MetricsUtil.LOG.error("Unexpected attribute suffix");
-        throw new AttributeNotFoundException();
-      }
-    } else {
-        MetricsUtil.LOG.error("unknown metrics type: " + o.getClass().getName());
-        throw new AttributeNotFoundException();
-    }
-  }
-
-  @Override
-  public AttributeList getAttributes(String[] attributeNames) {
-    if (attributeNames == null || attributeNames.length == 0) 
-      throw new IllegalArgumentException();
-    
-    updateMbeanInfoIfMetricsListChanged();
-    
-    AttributeList result = new AttributeList(attributeNames.length);
-    for (String iAttributeName : attributeNames) {
-      try {
-        Object value = getAttribute(iAttributeName);
-        result.add(new Attribute(iAttributeName, value));
-      } catch (Exception e) {
-        continue;
-      } 
-    }
-    return result;
-  }
-
-  @Override
-  public MBeanInfo getMBeanInfo() {
-    return mbeanInfo;
-  }
-
-  @Override
-  public Object invoke(String actionName, Object[] parms, String[] signature)
-      throws MBeanException, ReflectionException {
-    
-    if (actionName == null || actionName.isEmpty()) 
-      throw new IllegalArgumentException();
-    
-    
-    // Right now we support only one fixed operation (if it applies)
-    if (!(actionName.equals(RESET_ALL_MIN_MAX_OP)) || 
-        mbeanInfo.getOperations().length != 1) {
-      throw new ReflectionException(new NoSuchMethodException(actionName));
-    }
-    for (MetricsBase m : metricsRegistry.getMetricsList())  {
-      if ( MetricsTimeVaryingRate.class.isInstance(m) ) {
-        MetricsTimeVaryingRate.class.cast(m).resetMinMax();
-      }
-    }
-    return null;
-  }
-
-  @Override
-  public void setAttribute(Attribute attribute)
-      throws AttributeNotFoundException, InvalidAttributeValueException,
-      MBeanException, ReflectionException {
-    throw new ReflectionException(new NoSuchMethodException("set" + attribute));
-  }
-
-  @Override
-  public AttributeList setAttributes(AttributeList attributes) {
-    return null;
-  }
-}

+ 0 - 106
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsIntValue.java

@@ -1,106 +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.metrics.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * The MetricsIntValue class is for a metric that is not time varied
- * but changes only when it is set. 
- * Each time its value is set, it is published only *once* at the next update
- * call.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public class MetricsIntValue extends MetricsBase {  
-
-  private static final Log LOG =
-    LogFactory.getLog("org.apache.hadoop.metrics.util");
-
-  private int value;
-  private boolean changed;
-  
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   */
-  public MetricsIntValue(final String nam, final MetricsRegistry registry, final String description) {
-    super(nam, description);
-    value = 0;
-    changed = false;
-    registry.add(nam, this);
-  }
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   * A description of {@link #NO_DESCRIPTION} is used
-   */
-  public MetricsIntValue(final String nam, MetricsRegistry registry) {
-    this(nam, registry, NO_DESCRIPTION);
-  }
-  
-  
-  
-  /**
-   * Set the value
-   * @param newValue
-   */
-  public synchronized void set(final int newValue) {
-    value = newValue;
-    changed = true;
-  }
-  
-  /**
-   * Get value
-   * @return the value last set
-   */
-  public synchronized int get() { 
-    return value;
-  } 
-  
-
-  /**
-   * Push the metric to the mr.
-   * The metric is pushed only if it was updated since last push
-   * 
-   * Note this does NOT push to JMX
-   * (JMX gets the info via {@link #get()}
-   *
-   * @param mr
-   */
-  @Override
-  public synchronized void pushMetric(final MetricsRecord mr) {
-    if (changed) {
-      try {
-        mr.setMetric(getName(), value);
-      } catch (Exception e) {
-        LOG.info("pushMetric failed for " + getName() + "\n", e);
-      }
-    }
-    changed = false;
-  }
-}

+ 0 - 93
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsLongValue.java

@@ -1,93 +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.metrics.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.metrics.MetricsRecord;
-
-
-/**
- * The MetricsLongValue class is for a metric that is not time varied
- * but changes only when it is set. 
- * Each time its value is set, it is published only *once* at the next update
- * call.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public class MetricsLongValue extends MetricsBase{  
-  private long value;
-  private boolean changed;
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   */
-  public MetricsLongValue(final String nam, final MetricsRegistry registry, final String description) {
-    super(nam, description);
-    value = 0;
-    changed = false;
-    registry.add(nam, this);
-  }
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   * A description of {@link #NO_DESCRIPTION} is used
-   */
-  public MetricsLongValue(final String nam, MetricsRegistry registry) {
-    this(nam, registry, NO_DESCRIPTION);
-  }
-  
-  /**
-   * Set the value
-   * @param newValue
-   */
-  public synchronized void set(final long newValue) {
-    value = newValue;
-    changed = true;
-  }
-  
-  /**
-   * Get value
-   * @return the value last set
-   */
-  public synchronized long get() { 
-    return value;
-  } 
- 
-
-  /**
-   * Push the metric to the mr.
-   * The metric is pushed only if it was updated since last push
-   * 
-   * Note this does NOT push to JMX
-   * (JMX gets the info via {@link #get()}
-   *
-   * @param mr
-   */
-  @Override
-  public synchronized void pushMetric(final MetricsRecord mr) {
-    if (changed) 
-      mr.setMetric(getName(), value);
-    changed = false;
-  }
-}

+ 0 - 90
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsRegistry.java

@@ -1,90 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics.util;
-
-import java.util.Collection;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * 
- * This is the registry for metrics.
- * Related set of metrics should be declared in a holding class and registered
- * in a registry for those metrics which is also stored in the the holding class.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public class MetricsRegistry {
-  private ConcurrentHashMap<String, MetricsBase> metricsList =
-      new ConcurrentHashMap<String, MetricsBase>();
-
-  public MetricsRegistry() {
-  }
-  
-  /**
-   * 
-   * @return number of metrics in the registry
-   */
-  public int size() {
-    return metricsList.size();
-  }
-  
-  /**
-   * Add a new metrics to the registry
-   * @param metricsName - the name
-   * @param theMetricsObj - the metrics
-   * @throws IllegalArgumentException if a name is already registered
-   */
-  public void add(final String metricsName, final MetricsBase theMetricsObj) {
-    if (metricsList.putIfAbsent(metricsName, theMetricsObj) != null) {
-      throw new IllegalArgumentException("Duplicate metricsName:" +
-          metricsName);
-    }
-  }
-
-  
-  /**
-   * 
-   * @param metricsName
-   * @return the metrics if there is one registered by the supplied name.
-   *         Returns null if none is registered
-   */
-  public MetricsBase get(final String metricsName) {
-    return metricsList.get(metricsName);
-  }
-  
-  
-  /**
-   * 
-   * @return the list of metrics names
-   */
-  public Collection<String> getKeyList() {
-    return metricsList.keySet();
-  }
-  
-  /**
-   * 
-   * @return the list of metrics
-   */
-  public Collection<MetricsBase> getMetricsList() {
-    return metricsList.values();
-  }
-}

+ 0 - 129
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingInt.java

@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * The MetricsTimeVaryingInt class is for a metric that naturally
- * varies over time (e.g. number of files created). The metrics is accumulated
- * over an interval (set in the metrics config file); the metrics is
- *  published at the end of each interval and then 
- * reset to zero. Hence the counter has the value in the current interval. 
- * 
- * Note if one wants a time associated with the metric then use
- * @see org.apache.hadoop.metrics.util.MetricsTimeVaryingRate
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public class MetricsTimeVaryingInt extends MetricsBase {
-
-  private static final Log LOG =
-    LogFactory.getLog("org.apache.hadoop.metrics.util");
-  
-  private int currentValue;
-  private int previousIntervalValue;
-  
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   * @param description - the description
-   */
-  public MetricsTimeVaryingInt(final String nam,
-                               final MetricsRegistry registry,
-                               final String description) {
-    super(nam, description);
-    currentValue = 0;
-    previousIntervalValue = 0;
-    registry.add(nam, this);
-  }
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   * A description of {@link #NO_DESCRIPTION} is used
-   */
-  public MetricsTimeVaryingInt(final String nam, final MetricsRegistry registry) {
-    this(nam, registry, NO_DESCRIPTION);
-  }
-  
-
-  
-  /**
-   * Inc metrics for incr vlaue
-   * @param incr - number of operations
-   */
-  public synchronized void inc(final int incr) {
-    currentValue += incr;
-  }
-  
-  /**
-   * Inc metrics by one
-   */
-  public synchronized void inc() {
-    currentValue++;
-  }
-
-  private synchronized void intervalHeartBeat() {
-     previousIntervalValue = currentValue;
-     currentValue = 0;
-  }
-  
-  /**
-   * Push the delta  metrics to the mr.
-   * The delta is since the last push/interval.
-   * 
-   * Note this does NOT push to JMX
-   * (JMX gets the info via {@link #previousIntervalValue}
-   *
-   * @param mr
-   */
-  @Override
-  public synchronized void pushMetric(final MetricsRecord mr) {
-    intervalHeartBeat();
-    try {
-      mr.incrMetric(getName(), getPreviousIntervalValue());
-    } catch (Exception e) {
-      LOG.info("pushMetric failed for " + getName() + "\n" , e);
-    }
-  }
-  
-  
-  /**
-   * The Value at the Previous interval
-   * @return prev interval value
-   */
-  public synchronized int getPreviousIntervalValue() { 
-    return previousIntervalValue;
-  }
-  
-  /**
-   * The Value at the current interval
-   * @return prev interval value
-   */
-  public synchronized int getCurrentIntervalValue() { 
-    return currentValue;
-  } 
-}

+ 0 - 125
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingLong.java

@@ -1,125 +0,0 @@
-package org.apache.hadoop.metrics.util;
-
-/**
- * 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.
- */
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * The MetricsTimeVaryingLong class is for a metric that naturally
- * varies over time (e.g. number of files created). The metrics is accumulated
- * over an interval (set in the metrics config file); the metrics is
- *  published at the end of each interval and then 
- * reset to zero. Hence the counter has the value in the current interval. 
- * 
- * Note if one wants a time associated with the metric then use
- * @see org.apache.hadoop.metrics.util.MetricsTimeVaryingRate
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public class MetricsTimeVaryingLong extends MetricsBase{
-
-  private static final Log LOG =
-    LogFactory.getLog("org.apache.hadoop.metrics.util");
- 
-  private long currentValue;
-  private long previousIntervalValue;
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   */
-  public MetricsTimeVaryingLong(final String nam, MetricsRegistry registry, final String description) {
-    super(nam, description);
-    currentValue = 0;
-    previousIntervalValue = 0;
-    registry.add(nam, this);
-  }
-  
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   * A description of {@link #NO_DESCRIPTION} is used
-   */
-  public MetricsTimeVaryingLong(final String nam, MetricsRegistry registry) {
-    this(nam, registry, NO_DESCRIPTION);
-  }
-  
-  /**
-   * Inc metrics for incr vlaue
-   * @param incr - number of operations
-   */
-  public synchronized void inc(final long incr) {
-    currentValue += incr;
-  }
-  
-  /**
-   * Inc metrics by one
-   */
-  public synchronized void inc() {
-    currentValue++;
-  }
-
-  private synchronized void intervalHeartBeat() {
-     previousIntervalValue = currentValue;
-     currentValue = 0;
-  }
-  
-  /**
-   * Push the delta  metrics to the mr.
-   * The delta is since the last push/interval.
-   * 
-   * Note this does NOT push to JMX
-   * (JMX gets the info via {@link #previousIntervalValue}
-   *
-   * @param mr
-   */
-  @Override
-  public synchronized void pushMetric(final MetricsRecord mr) {
-    intervalHeartBeat();
-    try {
-      mr.incrMetric(getName(), getPreviousIntervalValue());
-    } catch (Exception e) {
-      LOG.info("pushMetric failed for " + getName() + "\n" , e);
-    }
-  }
-  
-  
-  /**
-   * The Value at the Previous interval
-   * @return prev interval value
-   */
-  public synchronized long getPreviousIntervalValue() { 
-    return previousIntervalValue;
-  } 
-  
-  /**
-   * The Value at the current interval
-   * @return prev interval value
-   */
-  public synchronized long getCurrentIntervalValue() { 
-    return currentValue;
-  } 
-}

+ 0 - 198
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsTimeVaryingRate.java

@@ -1,198 +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.metrics.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * The MetricsTimeVaryingRate class is for a rate based metric that
- * naturally varies over time (e.g. time taken to create a file).
- * The rate is averaged at each interval heart beat (the interval
- * is set in the metrics config file).
- * This class also keeps track of the min and max rates along with 
- * a method to reset the min-max.
- *
- * @deprecated Use org.apache.hadoop.metrics2 package instead.
- */
-@Deprecated
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-public class MetricsTimeVaryingRate extends MetricsBase {
-
-  private static final Log LOG =
-    LogFactory.getLog("org.apache.hadoop.metrics.util");
-
-  static class Metrics {
-    int numOperations = 0;
-    long time = 0;  // total time or average time
-
-    void set(final Metrics resetTo) {
-      numOperations = resetTo.numOperations;
-      time = resetTo.time;
-    }
-    
-    void reset() {
-      numOperations = 0;
-      time = 0;
-    }
-  }
-  
-  static class MinMax {
-    long minTime = -1;
-    long maxTime = 0;
-    
-    void set(final MinMax newVal) {
-      minTime = newVal.minTime;
-      maxTime = newVal.maxTime;
-    }
-    
-    void reset() {
-      minTime = -1;
-      maxTime = 0;
-    }
-    void update(final long time) { // update min max
-      minTime = (minTime == -1) ? time : Math.min(minTime, time);
-      minTime = Math.min(minTime, time);
-      maxTime = Math.max(maxTime, time);
-    }
-  }
-  private Metrics currentData;
-  private Metrics previousIntervalData;
-  private MinMax minMax;
-  
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   */
-  public MetricsTimeVaryingRate(final String nam, final MetricsRegistry registry, final String description) {
-    super(nam, description);
-    currentData = new Metrics();
-    previousIntervalData = new Metrics();
-    minMax = new MinMax();
-    registry.add(nam, this);
-  }
-  
-  /**
-   * Constructor - create a new metric
-   * @param nam the name of the metrics to be used to publish the metric
-   * @param registry - where the metrics object will be registered
-   * A description of {@link #NO_DESCRIPTION} is used
-   */
-  public MetricsTimeVaryingRate(final String nam, MetricsRegistry registry) {
-    this(nam, registry, NO_DESCRIPTION);
-
-  }
-  
-  
-  /**
-   * Increment the metrics for numOps operations
-   * @param numOps - number of operations
-   * @param time - time for numOps operations
-   */
-  public synchronized void inc(final int numOps, final long time) {
-    currentData.numOperations += numOps;
-    currentData.time += time;
-    long timePerOps = time/numOps;
-    minMax.update(timePerOps);
-  }
-  
-  /**
-   * Increment the metrics for one operation
-   * @param time for one operation
-   */
-  public synchronized void inc(final long time) {
-    currentData.numOperations++;
-    currentData.time += time;
-    minMax.update(time);
-  }
-  
-  
-
-  private synchronized void intervalHeartBeat() {
-     previousIntervalData.numOperations = currentData.numOperations;
-     previousIntervalData.time = (currentData.numOperations == 0) ?
-                             0 : currentData.time / currentData.numOperations;
-     currentData.reset();
-  }
-  
-  /**
-   * Push the delta  metrics to the mr.
-   * The delta is since the last push/interval.
-   * 
-   * Note this does NOT push to JMX
-   * (JMX gets the info via {@link #getPreviousIntervalAverageTime()} and
-   * {@link #getPreviousIntervalNumOps()}
-   *
-   * @param mr
-   */
-  @Override
-  public synchronized void pushMetric(final MetricsRecord mr) {
-    intervalHeartBeat();
-    try {
-      mr.incrMetric(getName() + "_num_ops", getPreviousIntervalNumOps());
-      mr.setMetric(getName() + "_avg_time", getPreviousIntervalAverageTime());
-    } catch (Exception e) {
-      LOG.info("pushMetric failed for " + getName() + "\n" , e);
-    }
-  }
-  
-  /**
-   * The number of operations in the previous interval
-   * @return - ops in prev interval
-   */
-  public synchronized int getPreviousIntervalNumOps() { 
-    return previousIntervalData.numOperations;
-  }
-  
-  /**
-   * The average rate of an operation in the previous interval
-   * @return - the average rate.
-   */
-  public synchronized long getPreviousIntervalAverageTime() {
-    return previousIntervalData.time;
-  } 
-  
-  /**
-   * The min time for a single operation since the last reset
-   *  {@link #resetMinMax()}
-   * @return min time for an operation
-   */
-  public synchronized long getMinTime() {
-    return  minMax.minTime;
-  }
-  
-  /**
-   * The max time for a single operation since the last reset
-   *  {@link #resetMinMax()}
-   * @return max time for an operation
-   */
-  public synchronized long getMaxTime() {
-    return minMax.maxTime;
-  }
-  
-  /**
-   * Reset the min max values
-   */
-  public synchronized void resetMinMax() {
-    minMax.reset();
-  }
-}

+ 0 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/package-info.java

@@ -1,22 +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.
- */
-@InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "MapReduce"})
-@InterfaceStability.Evolving
-package org.apache.hadoop.metrics.util;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;

+ 47 - 42
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java

@@ -23,7 +23,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.lang.reflect.Array;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.MalformedURLException;
@@ -40,7 +39,6 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.slf4j.Logger;
@@ -53,7 +51,7 @@ public class RunJar {
 
   private static final Logger LOG = LoggerFactory.getLogger(RunJar.class);
 
-  /** Pattern that matches any string */
+  /** Pattern that matches any string. */
   public static final Pattern MATCH_ANY = Pattern.compile(".*");
 
   /**
@@ -76,10 +74,21 @@ public class RunJar {
   public static final String HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES =
       "HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES";
 
+  /**
+   * Buffer size for copy the content of compressed file to new file.
+   */
+  private static final int BUFFER_SIZE = 8_192;
+
   /**
    * Unpack a jar file into a directory.
    *
    * This version unpacks all files inside the jar regardless of filename.
+   *
+   * @param jarFile the .jar file to unpack
+   * @param toDir the destination directory into which to unpack the jar
+   *
+   * @throws IOException if an I/O error has occurred or toDir
+   * cannot be created and does not already exist
    */
   public static void unJar(File jarFile, File toDir) throws IOException {
     unJar(jarFile, toDir, MATCH_ANY);
@@ -92,47 +101,43 @@ public class RunJar {
    * @param jarFile the .jar file to unpack
    * @param toDir the destination directory into which to unpack the jar
    * @param unpackRegex the pattern to match jar entries against
+   *
+   * @throws IOException if an I/O error has occurred or toDir
+   * cannot be created and does not already exist
    */
   public static void unJar(File jarFile, File toDir, Pattern unpackRegex)
-    throws IOException {
-    JarFile jar = new JarFile(jarFile);
-    try {
+      throws IOException {
+    try (JarFile jar = new JarFile(jarFile)) {
       int numOfFailedLastModifiedSet = 0;
       Enumeration<JarEntry> entries = jar.entries();
       while (entries.hasMoreElements()) {
         final JarEntry entry = entries.nextElement();
         if (!entry.isDirectory() &&
             unpackRegex.matcher(entry.getName()).matches()) {
-          InputStream in = jar.getInputStream(entry);
-          try {
+          try (InputStream in = jar.getInputStream(entry)) {
             File file = new File(toDir, entry.getName());
             ensureDirectory(file.getParentFile());
-            OutputStream out = new FileOutputStream(file);
-            try {
-              IOUtils.copyBytes(in, out, 8192);
-            } finally {
-              out.close();
+            try (OutputStream out = new FileOutputStream(file)) {
+              IOUtils.copyBytes(in, out, BUFFER_SIZE);
             }
             if (!file.setLastModified(entry.getTime())) {
               numOfFailedLastModifiedSet++;
             }
-          } finally {
-            in.close();
           }
         }
       }
       if (numOfFailedLastModifiedSet > 0) {
         LOG.warn("Could not set last modfied time for {} file(s)",
-                numOfFailedLastModifiedSet);
+            numOfFailedLastModifiedSet);
       }
-    } finally {
-      jar.close();
     }
   }
 
   /**
    * Ensure the existence of a given directory.
    *
+   * @param dir Directory to check
+   *
    * @throws IOException if it cannot be created and does not already exist
    */
   private static void ensureDirectory(File dir) throws IOException {
@@ -169,7 +174,7 @@ public class RunJar {
     JarFile jarFile;
     try {
       jarFile = new JarFile(fileName);
-    } catch(IOException io) {
+    } catch (IOException io) {
       throw new IOException("Error opening job jar: " + fileName)
         .initCause(io);
     }
@@ -193,11 +198,11 @@ public class RunJar {
     ensureDirectory(tmpDir);
 
     final File workDir;
-    try { 
+    try {
       workDir = File.createTempFile("hadoop-unjar", "", tmpDir);
     } catch (IOException ioe) {
-      // If user has insufficient perms to write to tmpDir, default  
-      // "Permission denied" message doesn't specify a filename. 
+      // If user has insufficient perms to write to tmpDir, default
+      // "Permission denied" message doesn't specify a filename.
       System.err.println("Error creating temp dir in java.io.tmpdir "
                          + tmpDir + " due to " + ioe.getMessage());
       System.exit(-1);
@@ -211,12 +216,12 @@ public class RunJar {
     ensureDirectory(workDir);
 
     ShutdownHookManager.get().addShutdownHook(
-      new Runnable() {
-        @Override
-        public void run() {
-          FileUtil.fullyDelete(workDir);
-        }
-      }, SHUTDOWN_HOOK_PRIORITY);
+        new Runnable() {
+          @Override
+          public void run() {
+            FileUtil.fullyDelete(workDir);
+          }
+        }, SHUTDOWN_HOOK_PRIORITY);
 
 
     unJar(file, workDir);
@@ -225,13 +230,13 @@ public class RunJar {
 
     Thread.currentThread().setContextClassLoader(loader);
     Class<?> mainClass = Class.forName(mainClassName, true, loader);
-    Method main = mainClass.getMethod("main", new Class[] {
-      Array.newInstance(String.class, 0).getClass()
-    });
-    String[] newArgs = Arrays.asList(args)
-      .subList(firstArg, args.length).toArray(new String[0]);
+    Method main = mainClass.getMethod("main", String[].class);
+    List<String> newArgsSubList = Arrays.asList(args)
+        .subList(firstArg, args.length);
+    String[] newArgs = newArgsSubList
+        .toArray(new String[newArgsSubList.size()]);
     try {
-      main.invoke(null, new Object[] { newArgs });
+      main.invoke(null, new Object[] {newArgs});
     } catch (InvocationTargetException e) {
       throw e.getTargetException();
     }
@@ -251,10 +256,10 @@ public class RunJar {
     // see if the client classloader is enabled
     if (useClientClassLoader()) {
       StringBuilder sb = new StringBuilder();
-      sb.append(workDir+"/").
+      sb.append(workDir).append("/").
           append(File.pathSeparator).append(file).
-          append(File.pathSeparator).append(workDir+"/classes/").
-          append(File.pathSeparator).append(workDir+"/lib/*");
+          append(File.pathSeparator).append(workDir).append("/classes/").
+          append(File.pathSeparator).append(workDir).append("/lib/*");
       // HADOOP_CLASSPATH is added to the client classpath
       String hadoopClasspath = getHadoopClasspath();
       if (hadoopClasspath != null && !hadoopClasspath.isEmpty()) {
@@ -270,18 +275,18 @@ public class RunJar {
       loader = new ApplicationClassLoader(clientClasspath,
           getClass().getClassLoader(), systemClassesList);
     } else {
-      List<URL> classPath = new ArrayList<URL>();
-      classPath.add(new File(workDir+"/").toURI().toURL());
+      List<URL> classPath = new ArrayList<>();
+      classPath.add(new File(workDir + "/").toURI().toURL());
       classPath.add(file.toURI().toURL());
       classPath.add(new File(workDir, "classes/").toURI().toURL());
       File[] libs = new File(workDir, "lib").listFiles();
       if (libs != null) {
-        for (int i = 0; i < libs.length; i++) {
-          classPath.add(libs[i].toURI().toURL());
+        for (File lib : libs) {
+          classPath.add(lib.toURI().toURL());
         }
       }
       // create a normal parent-delegating classloader
-      loader = new URLClassLoader(classPath.toArray(new URL[0]));
+      loader = new URLClassLoader(classPath.toArray(new URL[classPath.size()]));
     }
     return loader;
   }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java

@@ -59,7 +59,7 @@ public class SysInfoWindows extends SysInfo {
 
   @VisibleForTesting
   long now() {
-    return System.nanoTime();
+    return Time.monotonicNow();
   }
 
   void reset() {

+ 255 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationFieldsBase.java

@@ -124,6 +124,11 @@ public abstract class TestConfigurationFieldsBase {
    */
   private Map<String,String> configurationMemberVariables = null;
 
+  /**
+   * Member variable to store Configuration variables for later reference.
+   */
+  private Map<String,String> configurationDefaultVariables = null;
+
   /**
    * Member variable to store XML properties for later comparison.
    */
@@ -146,6 +151,7 @@ public abstract class TestConfigurationFieldsBase {
    */
   protected boolean configDebug = false;
   protected boolean xmlDebug = false;
+  protected boolean defaultDebug = false;
 
   /**
    * Abstract method to be used by subclasses for initializing base
@@ -316,6 +322,79 @@ public abstract class TestConfigurationFieldsBase {
     return retVal;
   }
 
+  /**
+   * Utility function to extract &quot;public static final&quot; default
+   * member variables from a Configuration type class.
+   *
+   * @param fields The class member variables
+   * @return HashMap containing <DefaultVariableName,DefaultValue> entries
+   */
+  private HashMap<String,String>
+      extractDefaultVariablesFromConfigurationFields(Field[] fields) {
+    // Sanity Check
+    if (fields==null) {
+      return null;
+    }
+
+    HashMap<String,String> retVal = new HashMap<String,String>();
+
+    // Setup regexp for valid properties
+    String propRegex = "^[A-Za-z][A-Za-z0-9_-]+(\\.[A-Za-z0-9_-]+)+$";
+    Pattern p = Pattern.compile(propRegex);
+
+    // Iterate through class member variables
+    int totalFields = 0;
+    String value;
+    for (Field f : fields) {
+      // Filter out anything that isn't "public static final"
+      if (!Modifier.isStatic(f.getModifiers()) ||
+          !Modifier.isPublic(f.getModifiers()) ||
+          !Modifier.isFinal(f.getModifiers())) {
+        continue;
+      }
+      // Special: Stuff any property beginning with "DEFAULT_" into a
+      // different hash for later processing
+      if (f.getName().startsWith("DEFAULT_") ||
+          f.getName().endsWith("_DEFAULT")) {
+        if (retVal.containsKey(f.getName())) {
+          continue;
+        }
+        try {
+          if (f.getType().getName().equals("java.lang.String")) {
+            String sValue = (String) f.get(null);
+            retVal.put(f.getName(),sValue);
+          } else if (f.getType().getName().equals("short")) {
+            short shValue = (short) f.get(null);
+            retVal.put(f.getName(),Integer.toString(shValue));
+          } else if (f.getType().getName().equals("int")) {
+            int iValue = (int) f.get(null);
+            retVal.put(f.getName(),Integer.toString(iValue));
+          } else if (f.getType().getName().equals("long")) {
+            long lValue = (long) f.get(null);
+            retVal.put(f.getName(),Long.toString(lValue));
+          } else if (f.getType().getName().equals("float")) {
+            float fValue = (float) f.get(null);
+            retVal.put(f.getName(),Float.toString(fValue));
+          } else if (f.getType().getName().equals("double")) {
+            double dValue = (double) f.get(null);
+            retVal.put(f.getName(),Double.toString(dValue));
+          } else if (f.getType().getName().equals("boolean")) {
+            boolean bValue = (boolean) f.get(null);
+            retVal.put(f.getName(),Boolean.toString(bValue));
+          } else {
+            if (defaultDebug) {
+              System.out.println("Config variable " + f.getName() + " has unknown type " + f.getType().getName());
+            }
+          }
+        } catch (IllegalAccessException iaException) {
+          iaException.printStackTrace();
+        }
+      }
+    }
+
+    return retVal;
+  }
+
   /**
    * Perform set difference operation on keyMap2 from keyMap1.
    *
@@ -374,6 +453,26 @@ public abstract class TestConfigurationFieldsBase {
       System.out.println("");
     }
 
+    // Create default configuration variable key/value map
+    if (defaultDebug) {
+      System.out.println("Reading Config property files for defaults");
+      System.out.println("");
+    }
+    configurationDefaultVariables = new HashMap<String,String>();
+    for (Class c : configurationClasses) {
+      Field[] fields = c.getDeclaredFields();
+      Map<String,String> defaultMap =
+          extractDefaultVariablesFromConfigurationFields(fields);
+      if (defaultMap!=null) {
+        configurationDefaultVariables.putAll(defaultMap);
+      }
+    }
+    if (defaultDebug) {
+      System.out.println("");
+      System.out.println("=====");
+      System.out.println("");
+    }
+
     // Find class members not in the XML file
     configurationFieldsMissingInXmlFile = compareConfigurationToXmlFields
         (configurationMemberVariables, xmlKeyValueMap);
@@ -464,4 +563,160 @@ public abstract class TestConfigurationFieldsBase {
       assertTrue(configErrorMsg.toString(), missingConfigSize==0);
     }
   }
+
+  /**
+   * For each property in the XML file, verify that the value matches
+   * up to the default if one exists.
+   */
+  @Test
+  public void testXmlAgainstDefaultValuesInConfigurationClass() {
+    // Error if subclass hasn't set class members
+    assertTrue(xmlFilename!=null);
+    assertTrue(configurationMemberVariables!=null);
+    assertTrue(configurationDefaultVariables!=null);
+
+    HashSet<String> xmlPropertiesWithEmptyValue = new HashSet<String>();
+    HashSet<String> configPropertiesWithNoDefaultConfig = new HashSet<String>();
+    HashMap<String,String> xmlPropertiesMatchingConfigDefault =
+        new HashMap<String,String>();
+    // Ugly solution.  Should have tuple-based solution.
+    HashMap<HashMap<String,String>,HashMap<String,String>> mismatchingXmlConfig =
+        new HashMap<HashMap<String,String>,HashMap<String,String>>();
+
+    for (Map.Entry<String,String> xEntry : xmlKeyValueMap.entrySet()) {
+      String xmlProperty = xEntry.getKey();
+      String xmlDefaultValue = xEntry.getValue();
+      String configProperty = configurationMemberVariables.get(xmlProperty);
+      if (configProperty!=null) {
+        String defaultConfigName = null;
+        String defaultConfigValue = null;
+
+        // Type 1: Prepend DEFAULT_
+        String defaultNameCheck1 = "DEFAULT_" + configProperty;
+        String defaultValueCheck1 = configurationDefaultVariables
+            .get(defaultNameCheck1);
+        // Type 2: Swap _KEY suffix with _DEFAULT suffix
+        String defaultNameCheck2 = null;
+        if (configProperty.endsWith("_KEY")) {
+          defaultNameCheck2 = configProperty
+              .substring(0,configProperty.length()-4) + "_DEFAULT";
+        }
+        String defaultValueCheck2 = configurationDefaultVariables
+            .get(defaultNameCheck2);
+        // Type Last: Append _DEFAULT suffix
+        String defaultNameCheck3 = configProperty + "_DEFAULT";
+        String defaultValueCheck3 = configurationDefaultVariables
+            .get(defaultNameCheck3);
+
+        // Pick the default value that exists
+        if (defaultValueCheck1!=null) {
+          defaultConfigName = defaultNameCheck1;
+          defaultConfigValue = defaultValueCheck1;
+        } else if (defaultValueCheck2!=null) {
+          defaultConfigName = defaultNameCheck2;
+          defaultConfigValue = defaultValueCheck2;
+        } else if (defaultValueCheck3!=null) {
+          defaultConfigName = defaultNameCheck3;
+          defaultConfigValue = defaultValueCheck3;
+        }
+
+        if (defaultConfigValue!=null) {
+          if (xmlDefaultValue==null) {
+            xmlPropertiesWithEmptyValue.add(xmlProperty);
+          } else if (!xmlDefaultValue.equals(defaultConfigValue)) {
+            HashMap<String,String> xmlEntry =
+                new HashMap<String,String>();
+            xmlEntry.put(xmlProperty,xmlDefaultValue);
+            HashMap<String,String> configEntry =
+                new HashMap<String,String>();
+            configEntry.put(defaultConfigName,defaultConfigValue);
+            mismatchingXmlConfig.put(xmlEntry,configEntry);
+           } else {
+            xmlPropertiesMatchingConfigDefault
+                .put(xmlProperty, defaultConfigName);
+          }
+        } else {
+          configPropertiesWithNoDefaultConfig.add(configProperty);
+        }
+      } else {
+      }
+    }
+
+    // Print out any unknown mismatching XML value/Config default value
+    System.out.println(this.xmlFilename + " has " +
+        mismatchingXmlConfig.size() +
+        " properties that do not match the default Config value");
+    if (mismatchingXmlConfig.size()==0) {
+      System.out.println("  (None)");
+    } else {
+       for (Map.Entry<HashMap<String,String>,HashMap<String,String>> xcEntry :
+           mismatchingXmlConfig.entrySet()) {
+         HashMap<String,String> xmlMap = xcEntry.getKey();
+         HashMap<String,String> configMap = xcEntry.getValue();
+         for (Map.Entry<String,String> xmlEntry : xmlMap.entrySet()) {
+           System.out.println("  XML Property: " + xmlEntry.getKey());
+           System.out.println("  XML Value:    " + xmlEntry.getValue());
+         }
+         for (Map.Entry<String,String> configEntry : configMap.entrySet()) {
+           System.out.println("  Config Name:  " + configEntry.getKey());
+           System.out.println("  Config Value: " + configEntry.getValue());
+         }
+         System.out.println("");
+       }
+    }
+    System.out.println();
+
+    // Print out Config properties that have no corresponding DEFAULT_*
+    // variable and cannot do any XML comparison (i.e. probably needs to
+    // be checked by hand)
+    System.out.println("Configuration(s) have " +
+        configPropertiesWithNoDefaultConfig.size() +
+        " properties with no corresponding default member variable.  These" +
+        " will need to be verified manually.");
+    if (configPropertiesWithNoDefaultConfig.size()==0) {
+      System.out.println("  (None)");
+    } else {
+      Iterator<String> cItr = configPropertiesWithNoDefaultConfig.iterator();
+      while (cItr.hasNext()) {
+        System.out.println("  " + cItr.next());
+      }
+    }
+    System.out.println();
+
+    // MAYBE TODO Print out any known mismatching XML value/Config default
+
+    // Print out XML properties that have empty values (i.e. should result
+    // in code-based default)
+    System.out.println(this.xmlFilename + " has " +
+        xmlPropertiesWithEmptyValue.size() + " properties with empty values");
+    if (xmlPropertiesWithEmptyValue.size()==0) {
+      System.out.println("  (None)");
+    } else {
+      Iterator<String> xItr = xmlPropertiesWithEmptyValue.iterator();
+      while (xItr.hasNext()) {
+        System.out.println("  " + xItr.next());
+      }
+    }
+    System.out.println();
+
+    // Print out any matching XML value/Config default value
+    System.out.println(this.xmlFilename + " has " +
+        xmlPropertiesMatchingConfigDefault.size() +
+        " properties which match a corresponding Config variable");
+    if (xmlPropertiesMatchingConfigDefault.size()==0) {
+      System.out.println("  (None)");
+    } else {
+      for (Map.Entry<String,String> xcEntry :
+          xmlPropertiesMatchingConfigDefault.entrySet()) {
+        System.out.println("  " + xcEntry.getKey() + " / " +
+            xcEntry.getValue());
+      }
+    }
+    System.out.println();
+
+    // Test separator
+    System.out.println();
+    System.out.println("=====");
+    System.out.println();
+  }
 }

+ 5 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -323,7 +323,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
   }
 
   /**
-   * Verify the access for /logs, /stacks, /conf, /logLevel and /metrics
+   * Verify the access for /logs, /stacks, /conf, and /logLevel
    * servlets, when authentication filters are set, but authorization is not
    * enabled.
    * @throws Exception 
@@ -349,7 +349,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     myServer.start();
     String serverURL = "http://" + NetUtils.getHostPortString(myServer.getConnectorAddress(0)) + "/";
     for (String servlet : new String[] { "conf", "logs", "stacks",
-        "logLevel", "metrics" }) {
+        "logLevel" }) {
       for (String user : new String[] { "userA", "userB" }) {
         assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(serverURL
             + servlet, user));
@@ -359,8 +359,8 @@ public class TestHttpServer extends HttpServerFunctionalTest {
   }
 
   /**
-   * Verify the administrator access for /logs, /stacks, /conf, /logLevel and
-   * /metrics servlets.
+   * Verify the administrator access for /logs, /stacks, /conf, and /logLevel
+   * servlets.
    * 
    * @throws Exception
    */
@@ -393,7 +393,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     String serverURL = "http://"
         + NetUtils.getHostPortString(myServer.getConnectorAddress(0)) + "/";
     for (String servlet : new String[] { "conf", "logs", "stacks",
-        "logLevel", "metrics" }) {
+        "logLevel" }) {
       for (String user : new String[] { "userA", "userB", "userC", "userD" }) {
         assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(serverURL
             + servlet, user));

+ 0 - 112
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics/TestMetricsServlet.java

@@ -1,112 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.metrics.MetricsServlet.TagsMetricsPair;
-import org.apache.hadoop.metrics.spi.NoEmitMetricsContext;
-import org.apache.hadoop.metrics.spi.OutputRecord;
-import org.mortbay.util.ajax.JSON;
-
-@Deprecated
-public class TestMetricsServlet extends TestCase {
-  MetricsContext nc1;
-  MetricsContext nc2;
-  // List containing nc1 and nc2.
-  List<MetricsContext> contexts;
-  OutputRecord outputRecord;
-  
-  /**
-   * Initializes, for testing, two NoEmitMetricsContext's, and adds one value 
-   * to the first of them.
-   */
-  @Override
-  public void setUp() throws IOException {
-    nc1 = new NoEmitMetricsContext();
-    nc1.init("test1", ContextFactory.getFactory());
-    nc2 = new NoEmitMetricsContext();
-    nc2.init("test2", ContextFactory.getFactory());
-    contexts = new ArrayList<MetricsContext>();
-    contexts.add(nc1);
-    contexts.add(nc2);
-
-    MetricsRecord r = nc1.createRecord("testRecord");
-    
-    r.setTag("testTag1", "testTagValue1");
-    r.setTag("testTag2", "testTagValue2");
-    r.setMetric("testMetric1", 1);
-    r.setMetric("testMetric2", 33);
-    r.update();
-
-    Map<String, Collection<OutputRecord>> m = nc1.getAllRecords();
-    assertEquals(1, m.size());
-    assertEquals(1, m.values().size());
-    Collection<OutputRecord> outputRecords = m.values().iterator().next();
-    assertEquals(1, outputRecords.size());
-    outputRecord = outputRecords.iterator().next();
-  }
-  
- 
-  
-  public void testTagsMetricsPair() throws IOException {
-    TagsMetricsPair pair = new TagsMetricsPair(outputRecord.getTagsCopy(), 
-        outputRecord.getMetricsCopy());
-    String s = JSON.toString(pair);
-    assertEquals(
-        "[{\"testTag1\":\"testTagValue1\",\"testTag2\":\"testTagValue2\"},"+
-        "{\"testMetric1\":1,\"testMetric2\":33}]", s);
-  }
-  
-  public void testGetMap() throws IOException {
-    MetricsServlet servlet = new MetricsServlet();
-    Map<String, Map<String, List<TagsMetricsPair>>> m = servlet.makeMap(contexts);
-    assertEquals("Map missing contexts", 2, m.size());
-    assertTrue(m.containsKey("test1"));
-   
-    Map<String, List<TagsMetricsPair>> m2 = m.get("test1");
-    
-    assertEquals("Missing records", 1, m2.size());
-    assertTrue(m2.containsKey("testRecord"));
-    assertEquals("Wrong number of tags-values pairs.", 1, m2.get("testRecord").size());
-  }
-  
-  public void testPrintMap() throws IOException {
-    StringWriter sw = new StringWriter();
-    PrintWriter out = new PrintWriter(sw);
-    MetricsServlet servlet = new MetricsServlet();
-    servlet.printMap(out, servlet.makeMap(contexts));
-    
-    String EXPECTED = "" +
-      "test1\n" +
-      "  testRecord\n" +
-      "    {testTag1=testTagValue1,testTag2=testTagValue2}:\n" +
-      "      testMetric1=1\n" +
-      "      testMetric2=33\n" +
-      "test2\n";
-    assertEquals(EXPECTED, sw.toString());
-  }
-}

+ 0 - 84
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics/ganglia/TestGangliaContext.java

@@ -1,84 +0,0 @@
-/*
- * TestGangliaContext.java
- *
- * 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.metrics.ganglia;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext;
-
-import java.net.MulticastSocket;
-
-@Deprecated
-public class TestGangliaContext {
-  @Test
-  public void testShouldCreateDatagramSocketByDefault() throws Exception {
-    GangliaContext context = new GangliaContext();
-    context.init("gangliaContext", ContextFactory.getFactory());
-    assertFalse("Created MulticastSocket", context.datagramSocket instanceof MulticastSocket);
-  }
-
-  @Test
-  public void testShouldCreateDatagramSocketIfMulticastIsDisabled() throws Exception {
-    GangliaContext context = new GangliaContext();
-    ContextFactory factory = ContextFactory.getFactory();
-    factory.setAttribute("gangliaContext.multicast", "false");
-    context.init("gangliaContext", factory);
-    assertFalse("Created MulticastSocket", context.datagramSocket instanceof MulticastSocket);
-  }
-
-  @Test
-  public void testShouldCreateMulticastSocket() throws Exception {
-    GangliaContext context = new GangliaContext();
-    ContextFactory factory = ContextFactory.getFactory();
-    factory.setAttribute("gangliaContext.multicast", "true");
-    context.init("gangliaContext", factory);
-    assertTrue("Did not create MulticastSocket", context.datagramSocket instanceof MulticastSocket);
-    MulticastSocket multicastSocket = (MulticastSocket) context.datagramSocket;
-    assertEquals("Did not set default TTL", multicastSocket.getTimeToLive(), 1);
-  }
-
-  @Test
-  public void testShouldSetMulticastSocketTtl() throws Exception {
-    GangliaContext context = new GangliaContext();
-    ContextFactory factory = ContextFactory.getFactory();
-    factory.setAttribute("gangliaContext.multicast", "true");
-    factory.setAttribute("gangliaContext.multicast.ttl", "10");
-    context.init("gangliaContext", factory);
-    MulticastSocket multicastSocket = (MulticastSocket) context.datagramSocket;
-    assertEquals("Did not set TTL", multicastSocket.getTimeToLive(), 10);
-  }
-  
-  @Test
-  public void testCloseShouldCloseTheSocketWhichIsCreatedByInit() throws Exception {
-    AbstractMetricsContext context=new GangliaContext();
-    context.init("gangliaContext", ContextFactory.getFactory());
-    GangliaContext gangliaContext =(GangliaContext) context;
-    assertFalse("Socket already closed",gangliaContext.datagramSocket.isClosed());
-    context.close();
-    assertTrue("Socket not closed",gangliaContext.datagramSocket.isClosed());
-  }
-}

+ 0 - 39
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics/spi/TestOutputRecord.java

@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.metrics.spi;
-
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
-import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
-
-import junit.framework.TestCase;
-
-@Deprecated
-public class TestOutputRecord extends TestCase {
-  public void testCopy() {
-    TagMap tags = new TagMap();
-    tags.put("tagkey", "tagval");
-    MetricMap metrics = new MetricMap();
-    metrics.put("metrickey", 123.4);
-    OutputRecord r = new OutputRecord(tags, metrics);
-    
-    assertEquals(tags, r.getTagsCopy());    
-    assertNotSame(tags, r.getTagsCopy());
-    assertEquals(metrics, r.getMetricsCopy());
-    assertNotSame(metrics, r.getMetricsCopy());
-  } 
-}

+ 33 - 29
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.util;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
@@ -29,15 +32,16 @@ import java.util.jar.JarOutputStream;
 import java.util.regex.Pattern;
 import java.util.zip.ZipEntry;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-public class TestRunJar extends TestCase {
+public class TestRunJar {
+  private static final String FOOBAR_TXT = "foobar.txt";
+  private static final String FOOBAZ_TXT = "foobaz.txt";
+  private static final int BUFF_SIZE = 2048;
   private File TEST_ROOT_DIR;
 
   private static final String TEST_JAR_NAME="test-runjar.jar";
@@ -45,9 +49,8 @@ public class TestRunJar extends TestCase {
   private static final long MOCKED_NOW = 1_460_389_972_000L;
   private static final long MOCKED_NOW_PLUS_TWO_SEC = MOCKED_NOW + 2_000;
 
-  @Override
   @Before
-  protected void setUp() throws Exception {
+  public void setUp() throws Exception {
     TEST_ROOT_DIR = GenericTestUtils.getTestDir(getClass().getSimpleName());
     if (!TEST_ROOT_DIR.exists()) {
       TEST_ROOT_DIR.mkdirs();
@@ -56,9 +59,8 @@ public class TestRunJar extends TestCase {
     makeTestJar();
   }
 
-  @Override
   @After
-  protected void tearDown() {
+  public void tearDown() {
     FileUtil.fullyDelete(TEST_ROOT_DIR);
   }
 
@@ -70,11 +72,11 @@ public class TestRunJar extends TestCase {
     File jarFile = new File(TEST_ROOT_DIR, TEST_JAR_NAME);
     JarOutputStream jstream =
         new JarOutputStream(new FileOutputStream(jarFile));
-    ZipEntry zipEntry1 = new ZipEntry("foobar.txt");
+    ZipEntry zipEntry1 = new ZipEntry(FOOBAR_TXT);
     zipEntry1.setTime(MOCKED_NOW);
     jstream.putNextEntry(zipEntry1);
     jstream.closeEntry();
-    ZipEntry zipEntry2 = new ZipEntry("foobaz.txt");
+    ZipEntry zipEntry2 = new ZipEntry(FOOBAZ_TXT);
     zipEntry2.setTime(MOCKED_NOW_PLUS_TWO_SEC);
     jstream.putNextEntry(zipEntry2);
     jstream.closeEntry();
@@ -86,50 +88,52 @@ public class TestRunJar extends TestCase {
    */
   @Test
   public void testUnJar() throws Exception {
-    File unjarDir = new File(TEST_ROOT_DIR, "unjar-all");
-    assertFalse("unjar dir shouldn't exist at test start",
-                new File(unjarDir, "foobar.txt").exists());
+    File unjarDir = getUnjarDir("unjar-all");
 
     // Unjar everything
     RunJar.unJar(new File(TEST_ROOT_DIR, TEST_JAR_NAME),
                  unjarDir);
     assertTrue("foobar unpacked",
-               new File(unjarDir, "foobar.txt").exists());
+               new File(unjarDir, TestRunJar.FOOBAR_TXT).exists());
     assertTrue("foobaz unpacked",
-               new File(unjarDir, "foobaz.txt").exists());
-
+               new File(unjarDir, FOOBAZ_TXT).exists());
   }
 
   /**
    * Test unjarring a specific regex
    */
+  @Test
   public void testUnJarWithPattern() throws Exception {
-    File unjarDir = new File(TEST_ROOT_DIR, "unjar-pattern");
-    assertFalse("unjar dir shouldn't exist at test start",
-                new File(unjarDir, "foobar.txt").exists());
+    File unjarDir = getUnjarDir("unjar-pattern");
 
     // Unjar only a regex
     RunJar.unJar(new File(TEST_ROOT_DIR, TEST_JAR_NAME),
                  unjarDir,
                  Pattern.compile(".*baz.*"));
     assertFalse("foobar not unpacked",
-                new File(unjarDir, "foobar.txt").exists());
+                new File(unjarDir, TestRunJar.FOOBAR_TXT).exists());
     assertTrue("foobaz unpacked",
-               new File(unjarDir, "foobaz.txt").exists());
-
+               new File(unjarDir, FOOBAZ_TXT).exists());
   }
 
+  @Test
   public void testUnJarDoesNotLooseLastModify() throws Exception {
-    File unjarDir = new File(TEST_ROOT_DIR, "unjar-lastmod");
-    assertFalse("unjar dir shouldn't exist at test start",
-            new File(unjarDir, "foobar.txt").exists());
+    File unjarDir = getUnjarDir("unjar-lastmod");
 
     // Unjar everything
     RunJar.unJar(new File(TEST_ROOT_DIR, TEST_JAR_NAME),
             unjarDir);
 
-    assertEquals("Last modify time was lost during unJar", MOCKED_NOW, new File(unjarDir, "foobar.txt").lastModified());
-    assertEquals("Last modify time was lost during unJar", MOCKED_NOW_PLUS_TWO_SEC, new File(unjarDir, "foobaz.txt").lastModified());
+    String failureMessage = "Last modify time was lost during unJar";
+    assertEquals(failureMessage, MOCKED_NOW, new File(unjarDir, TestRunJar.FOOBAR_TXT).lastModified());
+    assertEquals(failureMessage, MOCKED_NOW_PLUS_TWO_SEC, new File(unjarDir, FOOBAZ_TXT).lastModified());
+  }
+
+  private File getUnjarDir(String dirName) {
+    File unjarDir = new File(TEST_ROOT_DIR, dirName);
+    assertFalse("unjar dir shouldn't exist at test start",
+                new File(unjarDir, TestRunJar.FOOBAR_TXT).exists());
+    return unjarDir;
   }
 
   /**
@@ -174,10 +178,10 @@ public class TestRunJar extends TestCase {
       ZipEntry entry = new ZipEntry(name);
       jstream.putNextEntry(entry);
       BufferedInputStream bufInputStream = new BufferedInputStream(
-          entryInputStream, 2048);
+          entryInputStream, BUFF_SIZE);
       int count;
-      byte[] data = new byte[2048];
-      while ((count = bufInputStream.read(data, 0, 2048)) != -1) {
+      byte[] data = new byte[BUFF_SIZE];
+      while ((count = bufInputStream.read(data, 0, BUFF_SIZE)) != -1) {
         jstream.write(data, 0, count);
       }
       jstream.closeEntry();

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -2410,6 +2410,9 @@ public class DistributedFileSystem extends FileSystem {
       while (it.hasNext()) {
         Path ezTrashRoot = new Path(it.next().getPath(),
             FileSystem.TRASH_PREFIX);
+        if (!exists(ezTrashRoot)) {
+          continue;
+        }
         if (allUsers) {
           for (FileStatus candidate : listStatus(ezTrashRoot)) {
             if (exists(candidate.getPath())) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml

@@ -148,7 +148,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                     </exec>
                     <!-- Copy for inclusion in distribution. -->
                     <copy todir="${project.build.directory}/bin">
-                      <fileset dir="${project.build.directory}/native/target/bin/RelWithDebInfo"/>
+                      <fileset dir="${project.build.directory}/native/bin/RelWithDebInfo"/>
                     </copy>
                   </target>
                 </configuration>

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.CloseableReferenceCount;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Timer;
 import org.codehaus.jackson.annotate.JsonProperty;
@@ -118,9 +119,10 @@ public class FsVolumeImpl implements FsVolumeSpi {
       Configuration conf, StorageType storageType) throws IOException {
     this.dataset = dataset;
     this.storageID = storageID;
-    this.reserved = conf.getLong(
+    this.reserved = conf.getLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY
+        + "." + StringUtils.toLowerCase(storageType.toString()), conf.getLong(
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
-        DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT);
+        DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT));
     this.reservedForReplicas = new AtomicLong(0L);
     this.currentDir = currentDir;
     File parent = currentDir.getParentFile();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -321,6 +321,11 @@
   <name>dfs.datanode.du.reserved</name>
   <value>0</value>
   <description>Reserved space in bytes per volume. Always leave this much space free for non dfs use.
+      Specific storage type based reservation is also supported. The property can be followed with
+      corresponding storage types ([ssd]/[disk]/[archive]/[ram_disk]) for cluster with heterogeneous storage.
+      For example, reserved space for RAM_DISK storage can be configured using property
+      'dfs.datanode.du.reserved.ram_disk'. If specific storage type reservation is not configured
+      then dfs.datanode.du.reserved will be used.
   </description>
 </property>
 

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -28,6 +28,7 @@ import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
@@ -1442,6 +1443,43 @@ public class TestEncryptionZones {
     verifyShellDeleteWithTrash(shell, encFile);
   }
 
+  @Test(timeout = 120000)
+  public void testGetTrashRoots() throws Exception {
+    final HdfsAdmin dfsAdmin =
+        new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+    Path ezRoot1 = new Path("/ez1");
+    fs.mkdirs(ezRoot1);
+    dfsAdmin.createEncryptionZone(ezRoot1, TEST_KEY);
+    Path ezRoot2 = new Path("/ez2");
+    fs.mkdirs(ezRoot2);
+    dfsAdmin.createEncryptionZone(ezRoot2, TEST_KEY);
+    Path ezRoot3 = new Path("/ez3");
+    fs.mkdirs(ezRoot3);
+    dfsAdmin.createEncryptionZone(ezRoot3, TEST_KEY);
+    Collection<FileStatus> trashRootsBegin = fs.getTrashRoots(true);
+    assertEquals("Unexpected getTrashRoots result", 0, trashRootsBegin.size());
+
+    final Path encFile = new Path(ezRoot2, "encFile");
+    final int len = 8192;
+    DFSTestUtil.createFile(fs, encFile, len, (short) 1, 0xFEED);
+    Configuration clientConf = new Configuration(conf);
+    clientConf.setLong(FS_TRASH_INTERVAL_KEY, 1);
+    FsShell shell = new FsShell(clientConf);
+    verifyShellDeleteWithTrash(shell, encFile);
+
+    Collection<FileStatus> trashRootsDelete1 = fs.getTrashRoots(true);
+    assertEquals("Unexpected getTrashRoots result", 1,
+        trashRootsDelete1.size());
+
+    final Path nonEncFile = new Path("/nonEncFile");
+    DFSTestUtil.createFile(fs, nonEncFile, len, (short) 1, 0xFEED);
+    verifyShellDeleteWithTrash(shell, nonEncFile);
+
+    Collection<FileStatus> trashRootsDelete2 = fs.getTrashRoots(true);
+    assertEquals("Unexpected getTrashRoots result", 2,
+        trashRootsDelete2.size());
+  }
+
   private void verifyShellDeleteWithTrash(FsShell shell, Path path)
       throws Exception{
     try {

+ 35 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -36,7 +37,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
-
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
@@ -143,4 +144,37 @@ public class TestFsVolumeList {
     volumeList.addVolume(ref);
     assertNull(ref.getVolume());
   }
+
+  @Test
+  public void testDfsReservedForDifferentStorageTypes() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY, 100L);
+
+    File volDir = new File(baseDir, "volume-0");
+    volDir.mkdirs();
+    // when storage type reserved is not configured,should consider
+    // dfs.datanode.du.reserved.
+    FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", volDir, conf,
+        StorageType.RAM_DISK);
+    assertEquals("", 100L, volume.getReserved());
+    // when storage type reserved is configured.
+    conf.setLong(
+        DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY + "."
+            + StringUtils.toLowerCase(StorageType.RAM_DISK.toString()), 1L);
+    conf.setLong(
+        DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY + "."
+            + StringUtils.toLowerCase(StorageType.SSD.toString()), 2L);
+    FsVolumeImpl volume1 = new FsVolumeImpl(dataset, "storage-id", volDir,
+        conf, StorageType.RAM_DISK);
+    assertEquals("", 1L, volume1.getReserved());
+    FsVolumeImpl volume2 = new FsVolumeImpl(dataset, "storage-id", volDir,
+        conf, StorageType.SSD);
+    assertEquals("", 2L, volume2.getReserved());
+    FsVolumeImpl volume3 = new FsVolumeImpl(dataset, "storage-id", volDir,
+        conf, StorageType.DISK);
+    assertEquals("", 100L, volume3.getReserved());
+    FsVolumeImpl volume4 = new FsVolumeImpl(dataset, "storage-id", volDir,
+        conf, StorageType.DEFAULT);
+    assertEquals("", 100L, volume4.getReserved());
+  }
 }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -768,7 +768,7 @@ public class LocalJobRunner implements ClientProtocol {
   public LocalJobRunner(JobConf conf) throws IOException {
     this.fs = FileSystem.getLocal(conf);
     this.conf = conf;
-    myMetrics = new LocalJobRunnerMetrics(new JobConf(conf));
+    myMetrics = LocalJobRunnerMetrics.create();
   }
 
   // JobSubmissionProtocol methods

+ 31 - 63
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetrics.java

@@ -17,82 +17,50 @@
  */
 package org.apache.hadoop.mapred;
 
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.hadoop.metrics.jvm.JvmMetrics;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 
-@SuppressWarnings("deprecation")
-class LocalJobRunnerMetrics implements Updater {
-  private final MetricsRecord metricsRecord;
+import java.util.concurrent.ThreadLocalRandom;
 
-  private int numMapTasksLaunched = 0;
-  private int numMapTasksCompleted = 0;
-  private int numReduceTasksLaunched = 0;
-  private int numReduceTasksCompleted = 0;
-  private int numWaitingMaps = 0;
-  private int numWaitingReduces = 0;
-  
-  public LocalJobRunnerMetrics(JobConf conf) {
-    String sessionId = conf.getSessionId();
-    // Initiate JVM Metrics
-    JvmMetrics.init("JobTracker", sessionId);
-    // Create a record for map-reduce metrics
-    MetricsContext context = MetricsUtil.getContext("mapred");
-    // record name is jobtracker for compatibility 
-    metricsRecord = MetricsUtil.createRecord(context, "jobtracker");
-    metricsRecord.setTag("sessionId", sessionId);
-    context.registerUpdater(this);
+@Metrics(name="LocalJobRunnerMetrics", context="mapred")
+final class LocalJobRunnerMetrics {
+
+  @Metric
+  private MutableCounterInt numMapTasksLaunched;
+  @Metric
+  private MutableCounterInt numMapTasksCompleted;
+  @Metric
+  private MutableCounterInt numReduceTasksLaunched;
+  @Metric
+  private MutableGaugeInt numReduceTasksCompleted;
+
+  private LocalJobRunnerMetrics() {
   }
-    
-  /**
-   * Since this object is a registered updater, this method will be called
-   * periodically, e.g. every 5 seconds.
-   */
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      metricsRecord.incrMetric("maps_launched", numMapTasksLaunched);
-      metricsRecord.incrMetric("maps_completed", numMapTasksCompleted);
-      metricsRecord.incrMetric("reduces_launched", numReduceTasksLaunched);
-      metricsRecord.incrMetric("reduces_completed", numReduceTasksCompleted);
-      metricsRecord.incrMetric("waiting_maps", numWaitingMaps);
-      metricsRecord.incrMetric("waiting_reduces", numWaitingReduces);
 
-      numMapTasksLaunched = 0;
-      numMapTasksCompleted = 0;
-      numReduceTasksLaunched = 0;
-      numReduceTasksCompleted = 0;
-      numWaitingMaps = 0;
-      numWaitingReduces = 0;
-    }
-    metricsRecord.update();
+  public static LocalJobRunnerMetrics create() {
+    MetricsSystem ms = DefaultMetricsSystem.initialize("JobTracker");
+    return ms.register("LocalJobRunnerMetrics-" +
+            ThreadLocalRandom.current().nextInt(), null,
+        new LocalJobRunnerMetrics());
   }
 
   public synchronized void launchMap(TaskAttemptID taskAttemptID) {
-    ++numMapTasksLaunched;
-    decWaitingMaps(taskAttemptID.getJobID(), 1);
+    numMapTasksLaunched.incr();
   }
 
-  public synchronized void completeMap(TaskAttemptID taskAttemptID) {
-    ++numMapTasksCompleted;
+  public void completeMap(TaskAttemptID taskAttemptID) {
+    numMapTasksCompleted.incr();
   }
 
   public synchronized void launchReduce(TaskAttemptID taskAttemptID) {
-    ++numReduceTasksLaunched;
-    decWaitingReduces(taskAttemptID.getJobID(), 1);
-  }
-
-  public synchronized void completeReduce(TaskAttemptID taskAttemptID) {
-    ++numReduceTasksCompleted;
+    numReduceTasksLaunched.incr();
   }
 
-  private synchronized void decWaitingMaps(JobID id, int task) {
-    numWaitingMaps -= task;
-  }
-  
-  private synchronized void decWaitingReduces(JobID id, int task){
-    numWaitingReduces -= task;
+  public void completeReduce(TaskAttemptID taskAttemptID) {
+    numReduceTasksCompleted.incr();
   }
-
 }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java

@@ -68,7 +68,7 @@ public class Shuffle<K, V> implements ShuffleConsumerPlugin<K, V>, ExceptionRepo
     this.jobConf = context.getJobConf();
     this.umbilical = context.getUmbilical();
     this.reporter = context.getReporter();
-    this.metrics = new ShuffleClientMetrics(reduceId, jobConf);
+    this.metrics = ShuffleClientMetrics.create();
     this.copyPhase = context.getCopyPhase();
     this.taskStatus = context.getStatus();
     this.reduceTask = context.getReduceTask();

+ 37 - 54
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleClientMetrics.java

@@ -20,70 +20,53 @@ package org.apache.hadoop.mapreduce.task.reduce;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+
+import java.util.concurrent.ThreadLocalRandom;
 
 @InterfaceAudience.LimitedPrivate({"MapReduce"})
 @InterfaceStability.Unstable
-public class ShuffleClientMetrics implements Updater {
+@Metrics(name="ShuffleClientMetrics", context="mapred")
+public class ShuffleClientMetrics {
 
-  private MetricsRecord shuffleMetrics = null;
-  private int numFailedFetches = 0;
-  private int numSuccessFetches = 0;
-  private long numBytes = 0;
-  private int numThreadsBusy = 0;
-  private final int numCopiers;
-  
-  ShuffleClientMetrics(TaskAttemptID reduceId, JobConf jobConf) {
-    this.numCopiers = jobConf.getInt(MRJobConfig.SHUFFLE_PARALLEL_COPIES, 5);
+  @Metric
+  private MutableCounterInt numFailedFetches;
+  @Metric
+  private MutableCounterInt numSuccessFetches;
+  @Metric
+  private MutableCounterLong numBytes;
+  @Metric
+  private MutableGaugeInt numThreadsBusy;
 
-    MetricsContext metricsContext = MetricsUtil.getContext("mapred");
-    this.shuffleMetrics = 
-      MetricsUtil.createRecord(metricsContext, "shuffleInput");
-    this.shuffleMetrics.setTag("user", jobConf.getUser());
-    this.shuffleMetrics.setTag("jobName", jobConf.getJobName());
-    this.shuffleMetrics.setTag("jobId", reduceId.getJobID().toString());
-    this.shuffleMetrics.setTag("taskId", reduceId.toString());
-    this.shuffleMetrics.setTag("sessionId", jobConf.getSessionId());
-    metricsContext.registerUpdater(this);
+  private ShuffleClientMetrics() {
   }
-  public synchronized void inputBytes(long numBytes) {
-    this.numBytes += numBytes;
+
+  public static ShuffleClientMetrics create() {
+    MetricsSystem ms = DefaultMetricsSystem.initialize("JobTracker");
+    return ms.register("ShuffleClientMetrics-" +
+        ThreadLocalRandom.current().nextInt(), null,
+        new ShuffleClientMetrics());
   }
-  public synchronized void failedFetch() {
-    ++numFailedFetches;
+
+  public void inputBytes(long bytes) {
+    numBytes.incr(bytes);
   }
-  public synchronized void successFetch() {
-    ++numSuccessFetches;
+  public void failedFetch() {
+    numFailedFetches.incr();
   }
-  public synchronized void threadBusy() {
-    ++numThreadsBusy;
+  public void successFetch() {
+    numSuccessFetches.incr();
   }
-  public synchronized void threadFree() {
-    --numThreadsBusy;
+  public void threadBusy() {
+    numThreadsBusy.incr();
   }
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      shuffleMetrics.incrMetric("shuffle_input_bytes", numBytes);
-      shuffleMetrics.incrMetric("shuffle_failed_fetches", 
-                                numFailedFetches);
-      shuffleMetrics.incrMetric("shuffle_success_fetches", 
-                                numSuccessFetches);
-      if (numCopiers != 0) {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent",
-            100*((float)numThreadsBusy/numCopiers));
-      } else {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent", 0);
-      }
-      numBytes = 0;
-      numSuccessFetches = 0;
-      numFailedFetches = 0;
-    }
-    shuffleMetrics.update();
+  public void threadFree() {
+    numThreadsBusy.decr();
   }
 }

+ 18 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java

@@ -216,7 +216,16 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         String pID = entry.getKey();
         if (!pID.equals("1")) {
           ProcessInfo pInfo = entry.getValue();
-          ProcessInfo parentPInfo = allProcessInfo.get(pInfo.getPpid());
+          String ppid = pInfo.getPpid();
+          // If parent is init and process is not session leader,
+          // attach to sessionID
+          if (ppid.equals("1")) {
+              String sid = pInfo.getSessionId().toString();
+              if (!pID.equals(sid)) {
+                 ppid = sid;
+              }
+          }
+          ProcessInfo parentPInfo = allProcessInfo.get(ppid);
           if (parentPInfo != null) {
             parentPInfo.addChild(pInfo);
           }
@@ -571,6 +580,14 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     return pTree.substring(0, pTree.length()) + "]";
   }
 
+/**
+ * Returns boolean indicating whether pid
+ * is in process tree.
+ */
+  public boolean contains(String pid) {
+    return processTree.containsKey(pid);
+  }
+
   /**
    *
    * Class containing information of a process.

+ 23 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java

@@ -36,6 +36,7 @@ import java.util.Vector;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -65,7 +66,7 @@ public class TestProcfsBasedProcessTree {
     TestProcfsBasedProcessTree.class.getName() + "-localDir");
 
   private ShellCommandExecutor shexec = null;
-  private String pidFile, lowestDescendant;
+  private String pidFile, lowestDescendant, lostDescendant;
   private String shellScript;
 
   private static final int N = 6; // Controls the RogueTask
@@ -144,19 +145,17 @@ public class TestProcfsBasedProcessTree {
 
     lowestDescendant =
         TEST_ROOT_DIR + File.separator + "lowestDescendantPidFile";
+    lostDescendant =
+        TEST_ROOT_DIR + File.separator + "lostDescendantPidFile";
 
     // write to shell-script
-    try {
-      FileWriter fWriter = new FileWriter(shellScript);
-      fWriter.write("# rogue task\n" + "sleep 1\n" + "echo hello\n"
-          + "if [ $1 -ne 0 ]\n" + "then\n" + " sh " + shellScript
-          + " $(($1-1))\n" + "else\n" + " echo $$ > " + lowestDescendant + "\n"
-          + " while true\n do\n" + "  sleep 5\n" + " done\n" + "fi");
-      fWriter.close();
-    } catch (IOException ioe) {
-      LOG.info("Error: " + ioe);
-      return;
-    }
+    File file = new File(shellScript);
+    FileUtils.writeStringToFile(file, "# rogue task\n" + "sleep 1\n" + "echo hello\n"
+        + "if [ $1 -ne 0 ]\n" + "then\n" + " sh " + shellScript
+        + " $(($1-1))\n" + "else\n" + " echo $$ > " + lowestDescendant + "\n"
+        + "(sleep 300&\n"
+        + "echo $! > " + lostDescendant + ")\n"
+        + " while true\n do\n" + "  sleep 5\n" + " done\n" + "fi");
 
     Thread t = new RogueTaskThread();
     t.start();
@@ -179,6 +178,12 @@ public class TestProcfsBasedProcessTree {
     p.updateProcessTree(); // reconstruct
     LOG.info("ProcessTree: " + p.toString());
 
+    // Verify the orphaned pid is In process tree
+    String lostpid = getPidFromPidFile(lostDescendant);
+    LOG.info("Orphaned pid: " + lostpid);
+    Assert.assertTrue("Child process owned by init escaped process tree.",
+       p.contains(lostpid));
+
     // Get the process-tree dump
     String processTreeDump = p.getProcessTreeDump();
 
@@ -229,10 +234,12 @@ public class TestProcfsBasedProcessTree {
     Assert.assertFalse("ProcessTree must have been gone", isAlive(pid));
     Assert.assertTrue(
       "vmem for the gone-process is " + p.getVirtualMemorySize()
-          + " . It should be zero.", p.getVirtualMemorySize() == 0);
+          + " . It should be UNAVAILABLE(-1).",
+          p.getVirtualMemorySize() == UNAVAILABLE);
     Assert.assertTrue(
       "vmem (old API) for the gone-process is " + p.getCumulativeVmem()
-          + " . It should be zero.", p.getCumulativeVmem() == 0);
+          + " . It should be UNAVAILABLE(-1).",
+          p.getCumulativeVmem() == UNAVAILABLE);
     Assert.assertTrue(p.toString().equals("[ ]"));
   }
 
@@ -246,7 +253,7 @@ public class TestProcfsBasedProcessTree {
   }
 
   protected void destroyProcessTree(String pid) throws IOException {
-    sendSignal(pid, 9);
+    sendSignal("-"+pid, 9);
   }
 
   /**
@@ -910,7 +917,7 @@ public class TestProcfsBasedProcessTree {
 
   private static void sendSignal(String pid, int signal) throws IOException {
     ShellCommandExecutor shexec = null;
-    String[] arg = { "kill", "-" + signal, pid };
+    String[] arg = { "kill", "-" + signal, "--", pid };
     shexec = new ShellCommandExecutor(arg);
     shexec.execute();
   }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -1620,7 +1620,7 @@ public class ClientRMService extends AbstractService implements
     if (application == null) {
       RMAuditLogger.logFailure(callerUGI.getUserName(),
           AuditConstants.SIGNAL_CONTAINER, "UNKNOWN", "ClientRMService",
-          "Trying to signal an absent container", applicationId, containerId);
+          "Trying to signal an absent container", applicationId, containerId, null);
       throw RPCUtil
           .getRemoteException("Trying to signal an absent container "
               + containerId);
@@ -1644,11 +1644,11 @@ public class ClientRMService extends AbstractService implements
               request));
       RMAuditLogger.logSuccess(callerUGI.getShortUserName(),
           AuditConstants.SIGNAL_CONTAINER, "ClientRMService", applicationId,
-          containerId);
+          containerId, null);
     } else {
       RMAuditLogger.logFailure(callerUGI.getUserName(),
           AuditConstants.SIGNAL_CONTAINER, "UNKNOWN", "ClientRMService",
-          "Trying to signal an absent container", applicationId, containerId);
+          "Trying to signal an absent container", applicationId, containerId, null);
       throw RPCUtil
           .getRemoteException("Trying to signal an absent container "
               + containerId);

+ 28 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 
 /** 
  * Manages ResourceManager audit logs. 
@@ -38,7 +39,7 @@ public class RMAuditLogger {
 
   static enum Keys {USER, OPERATION, TARGET, RESULT, IP, PERMISSIONS,
                     DESCRIPTION, APPID, APPATTEMPTID, CONTAINERID, 
-                    CALLERCONTEXT, CALLERSIGNATURE}
+                    CALLERCONTEXT, CALLERSIGNATURE, RESOURCE}
 
   public static class AuditConstants {
     static final String SUCCESS = "SUCCESS";
@@ -77,9 +78,9 @@ public class RMAuditLogger {
   
   static String createSuccessLog(String user, String operation, String target,
       ApplicationId appId, ApplicationAttemptId attemptId,
-      ContainerId containerId) {
+      ContainerId containerId, Resource resource) {
     return createSuccessLog(user, operation, target, appId, attemptId,
-        containerId, null);
+        containerId, resource, null);
   }
 
   /**
@@ -87,7 +88,7 @@ public class RMAuditLogger {
    */
   static String createSuccessLog(String user, String operation, String target,
       ApplicationId appId, ApplicationAttemptId attemptId,
-      ContainerId containerId, CallerContext callerContext) {
+      ContainerId containerId, Resource resource, CallerContext callerContext) {
     StringBuilder b = new StringBuilder();
     start(Keys.USER, user, b);
     addRemoteIP(b);
@@ -103,6 +104,9 @@ public class RMAuditLogger {
     if (containerId != null) {
       add(Keys.CONTAINERID, containerId.toString(), b);
     }
+    if (resource != null) {
+      add(Keys.RESOURCE, resource.toString(), b);
+    }
     appendCallerContext(b, callerContext);
     return b.toString();
   }
@@ -138,16 +142,17 @@ public class RMAuditLogger {
    * @param target The target on which the operation is being performed. 
    * @param appId Application Id in which operation was performed.
    * @param containerId Container Id in which operation was performed.
+   * @param resource Resource associated with container.
    *
    * <br><br>
    * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
    * and hence the value fields should not contains tabs ('\t').
    */
   public static void logSuccess(String user, String operation, String target, 
-      ApplicationId appId, ContainerId containerId) {
+      ApplicationId appId, ContainerId containerId, Resource resource) {
     if (LOG.isInfoEnabled()) {
       LOG.info(createSuccessLog(user, operation, target, appId, null, 
-          containerId));
+          containerId, resource));
     }
   }
 
@@ -168,7 +173,7 @@ public class RMAuditLogger {
       ApplicationId appId, ApplicationAttemptId attemptId) {
     if (LOG.isInfoEnabled()) {
       LOG.info(createSuccessLog(user, operation, target, appId, attemptId,
-          null));
+          null, null));
     }
   }
   
@@ -176,7 +181,7 @@ public class RMAuditLogger {
       ApplicationId appId, CallerContext callerContext) {
     if (LOG.isInfoEnabled()) {
       LOG.info(createSuccessLog(user, operation, target, appId, null, null,
-          callerContext));
+          null, callerContext));
     }
   }
 
@@ -196,7 +201,7 @@ public class RMAuditLogger {
   public static void logSuccess(String user, String operation, String target,
       ApplicationId appId) {
     if (LOG.isInfoEnabled()) {
-      LOG.info(createSuccessLog(user, operation, target, appId, null, null));
+      LOG.info(createSuccessLog(user, operation, target, appId, null, null, null));
     }
   }
 
@@ -213,14 +218,14 @@ public class RMAuditLogger {
    */
   public static void logSuccess(String user, String operation, String target) {
     if (LOG.isInfoEnabled()) {
-      LOG.info(createSuccessLog(user, operation, target, null, null, null));
+      LOG.info(createSuccessLog(user, operation, target, null, null, null, null));
     }
   }
   
   static String createFailureLog(String user, String operation, String perm,
       String target, String description, ApplicationId appId,
       ApplicationAttemptId attemptId, ContainerId containerId,
-      CallerContext callerContext) {
+      Resource resource, CallerContext callerContext) {
     StringBuilder b = new StringBuilder();
     start(Keys.USER, user, b);
     addRemoteIP(b);
@@ -238,6 +243,9 @@ public class RMAuditLogger {
     if (containerId != null) {
       add(Keys.CONTAINERID, containerId.toString(), b);
     }
+    if (resource != null) {
+      add(Keys.RESOURCE, resource.toString(), b);
+    }
     appendCallerContext(b, callerContext);
     return b.toString();
   }
@@ -247,9 +255,9 @@ public class RMAuditLogger {
    */
   static String createFailureLog(String user, String operation, String perm,
       String target, String description, ApplicationId appId,
-      ApplicationAttemptId attemptId, ContainerId containerId) {
+      ApplicationAttemptId attemptId, ContainerId containerId, Resource resource) {
     return createFailureLog(user, operation, perm, target, description, appId,
-        attemptId, containerId, null);
+        attemptId, containerId, resource, null);
   }
 
   /**
@@ -263,6 +271,7 @@ public class RMAuditLogger {
    *                    failed.
    * @param appId Application Id in which operation was performed.
    * @param containerId Container Id in which operation was performed.
+   * @param resource Resources associated with container.
    *
    * <br><br>
    * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
@@ -270,10 +279,10 @@ public class RMAuditLogger {
    */
   public static void logFailure(String user, String operation, String perm,
       String target, String description, ApplicationId appId, 
-      ContainerId containerId) {
+      ContainerId containerId, Resource resource) {
     if (LOG.isWarnEnabled()) {
       LOG.warn(createFailureLog(user, operation, perm, target, description,
-          appId, null, containerId));
+          appId, null, containerId, resource));
     }
   }
 
@@ -297,7 +306,7 @@ public class RMAuditLogger {
       ApplicationAttemptId attemptId) {
     if (LOG.isWarnEnabled()) {
       LOG.warn(createFailureLog(user, operation, perm, target, description,
-          appId, attemptId, null));
+          appId, attemptId, null, null));
     }
   }
   
@@ -306,7 +315,7 @@ public class RMAuditLogger {
       CallerContext callerContext) {
     if (LOG.isWarnEnabled()) {
       LOG.warn(createFailureLog(user, operation, perm, target, description,
-          appId, null, null, callerContext));
+          appId, null, null, null, callerContext));
     }
   }
 
@@ -329,7 +338,7 @@ public class RMAuditLogger {
       String target, String description, ApplicationId appId) {
     if (LOG.isWarnEnabled()) {
       LOG.warn(createFailureLog(user, operation, perm, target, description,
-          appId, null, null));
+          appId, null, null, null));
     }
   }
 
@@ -351,7 +360,7 @@ public class RMAuditLogger {
       String target, String description) {
     if (LOG.isWarnEnabled()) {
       LOG.warn(createFailureLog(user, operation, perm, target, description,
-          null, null, null));
+          null, null, null, null));
     }
   }
 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java

@@ -503,7 +503,7 @@ public abstract class AbstractYarnScheduler
                 "Unauthorized access or invalid container", "Scheduler",
                 "Trying to release container not owned by app "
                     + "or with invalid id.", attempt.getApplicationId(),
-                containerId);
+                containerId, null);
           }
           attempt.getPendingRelease().clear();
         }
@@ -554,7 +554,7 @@ public abstract class AbstractYarnScheduler
             AuditConstants.RELEASE_CONTAINER,
             "Unauthorized access or invalid container", "Scheduler",
             "Trying to release container not owned by app or with invalid id.",
-            attempt.getApplicationId(), containerId);
+            attempt.getApplicationId(), containerId, null);
         }
       }
       completedContainer(rmContainer,

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java

@@ -165,12 +165,12 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     containersToPreempt.remove(containerId);
 
+    Resource containerResource = rmContainer.getContainer().getResource();
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId);
+        getApplicationId(), containerId, containerResource);
     
     // Update usage metrics 
-    Resource containerResource = rmContainer.getContainer().getResource();
     queue.getMetrics().releaseResources(getUser(), 1, containerResource);
     attemptResourceUsage.decUsed(partition, containerResource);
 
@@ -229,7 +229,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
     RMAuditLogger.logSuccess(getUser(),
         AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId);
+        getApplicationId(), containerId, container.getResource());
     
     return rmContainer;
   }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java

@@ -146,12 +146,12 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     // Remove from the list of containers
     liveContainers.remove(rmContainer.getContainerId());
 
+    Resource containerResource = rmContainer.getContainer().getResource();
     RMAuditLogger.logSuccess(getUser(), 
         AuditConstants.RELEASE_CONTAINER, "SchedulerApp", 
-        getApplicationId(), containerId);
+        getApplicationId(), containerId, containerResource);
     
     // Update usage metrics 
-    Resource containerResource = rmContainer.getContainer().getResource();
     queue.getMetrics().releaseResources(getUser(), 1, containerResource);
     this.attemptResourceUsage.decUsed(containerResource);
 
@@ -403,7 +403,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     }
     RMAuditLogger.logSuccess(getUser(), 
         AuditConstants.ALLOC_CONTAINER, "SchedulerApp", 
-        getApplicationId(), container.getId());
+        getApplicationId(), container.getId(), container.getResource());
     
     return rmContainer;
   }

+ 27 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys;
 import org.junit.Assert;
 import org.junit.Before;
@@ -61,6 +62,7 @@ public class TestRMAuditLogger {
   private static final ApplicationId APPID = mock(ApplicationId.class);
   private static final ApplicationAttemptId ATTEMPTID = mock(ApplicationAttemptId.class);
   private static final ContainerId CONTAINERID = mock(ContainerId.class);
+  private static final Resource RESOURCE = mock(Resource.class);
   private static final String CALLER_CONTEXT = "context";
   private static final byte[] CALLER_SIGNATURE = "signature".getBytes();
 
@@ -69,6 +71,7 @@ public class TestRMAuditLogger {
     when(APPID.toString()).thenReturn("app_1");
     when(ATTEMPTID.toString()).thenReturn("app_attempt_1");
     when(CONTAINERID.toString()).thenReturn("container_1");
+    when(RESOURCE.toString()).thenReturn("<memory:1536, vcores:1>");
   }
 
 
@@ -99,10 +102,11 @@ public class TestRMAuditLogger {
     expLog.append("\tTARGET=tgt");
     assertEquals(expLog.toString(), actLog.toString());
   }
-  
+
   private void testSuccessLogFormatHelper(boolean checkIP, ApplicationId appId,
       ApplicationAttemptId attemptId, ContainerId containerId) {
-    testSuccessLogFormatHelper(checkIP, appId, attemptId, containerId, null);
+    testSuccessLogFormatHelper(checkIP, appId, attemptId, containerId, null,
+        null);
   }
 
   /**
@@ -110,9 +114,9 @@ public class TestRMAuditLogger {
    */
   private void testSuccessLogFormatHelper(boolean checkIP, ApplicationId appId,
       ApplicationAttemptId attemptId, ContainerId containerId,
-      CallerContext callerContext) {
+      CallerContext callerContext, Resource resource) {
     String sLog = RMAuditLogger.createSuccessLog(USER, OPERATION, TARGET,
-        appId, attemptId, containerId, callerContext);
+        appId, attemptId, containerId, resource, callerContext);
     StringBuilder expLog = new StringBuilder();
     expLog.append("USER=test\t");
     if (checkIP) {
@@ -130,6 +134,9 @@ public class TestRMAuditLogger {
     if (containerId != null) {
       expLog.append("\tCONTAINERID=container_1");
     }
+    if (resource != null) {
+      expLog.append("\tRESOURCE=<memory:1536, vcores:1>");
+    }
     if (callerContext != null) {
       if (callerContext.getContext() != null) {
         expLog.append("\tCALLERCONTEXT=context");
@@ -146,7 +153,7 @@ public class TestRMAuditLogger {
    */
   private void testSuccessLogNulls(boolean checkIP) {
     String sLog = RMAuditLogger.createSuccessLog(null, null, null, null, 
-        null, null);
+        null, null, null);
     StringBuilder expLog = new StringBuilder();
     expLog.append("USER=null\t");
     if (checkIP) {
@@ -170,22 +177,22 @@ public class TestRMAuditLogger {
     testSuccessLogFormatHelper(checkIP, APPID, null, CONTAINERID);
     testSuccessLogFormatHelper(checkIP, null, ATTEMPTID, CONTAINERID);
     testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID);
-    testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID, null);
+    testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID, null, null);
     testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
-        new CallerContext.Builder(null).setSignature(null).build());
+        new CallerContext.Builder(null).setSignature(null).build(), RESOURCE);
     testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
-        new CallerContext.Builder(CALLER_CONTEXT).setSignature(null).build());
+        new CallerContext.Builder(CALLER_CONTEXT).setSignature(null).build(), RESOURCE);
     testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
-        new CallerContext.Builder(null).setSignature(CALLER_SIGNATURE).build());
+        new CallerContext.Builder(null).setSignature(CALLER_SIGNATURE).build(), RESOURCE);
     testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
         new CallerContext.Builder(CALLER_CONTEXT).setSignature(CALLER_SIGNATURE)
-            .build());
+            .build(), RESOURCE);
     testSuccessLogNulls(checkIP);
   }
 
   private void testFailureLogFormatHelper(boolean checkIP, ApplicationId appId,
       ApplicationAttemptId attemptId, ContainerId containerId) {
-    testFailureLogFormatHelper(checkIP, appId, attemptId, containerId, null);
+    testFailureLogFormatHelper(checkIP, appId, attemptId, containerId, null, null);
   }
  
   /**
@@ -193,10 +200,10 @@ public class TestRMAuditLogger {
    */
   private void testFailureLogFormatHelper(boolean checkIP, ApplicationId appId,
       ApplicationAttemptId attemptId, ContainerId containerId,
-      CallerContext callerContext) {
+      CallerContext callerContext, Resource resource) {
     String fLog =
       RMAuditLogger.createFailureLog(USER, OPERATION, PERM, TARGET, DESC,
-      appId, attemptId, containerId, callerContext);
+      appId, attemptId, containerId, resource, callerContext);
     StringBuilder expLog = new StringBuilder();
     expLog.append("USER=test\t");
     if (checkIP) {
@@ -215,6 +222,9 @@ public class TestRMAuditLogger {
     if (containerId != null) {
       expLog.append("\tCONTAINERID=container_1");
     }
+    if (resource != null) {
+      expLog.append("\tRESOURCE=<memory:1536, vcores:1>");
+    }
     if (callerContext != null) {
       if (callerContext.getContext() != null) {
         expLog.append("\tCALLERCONTEXT=context");
@@ -241,14 +251,14 @@ public class TestRMAuditLogger {
     testFailureLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID);
     
     testFailureLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
-        new CallerContext.Builder(null).setSignature(null).build());
+        new CallerContext.Builder(null).setSignature(null).build(), RESOURCE);
     testFailureLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
-        new CallerContext.Builder(CALLER_CONTEXT).setSignature(null).build());
+        new CallerContext.Builder(CALLER_CONTEXT).setSignature(null).build(), RESOURCE);
     testFailureLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
-        new CallerContext.Builder(null).setSignature(CALLER_SIGNATURE).build());
+        new CallerContext.Builder(null).setSignature(CALLER_SIGNATURE).build(), RESOURCE);
     testFailureLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID,
         new CallerContext.Builder(CALLER_CONTEXT).setSignature(CALLER_SIGNATURE)
-            .build());
+            .build(), RESOURCE);
   }
 
   /**

+ 125 - 131
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java

@@ -34,17 +34,22 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStartedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
@@ -130,24 +135,21 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       Exception {
 
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.NEW);
-
-    MockNM nm3 = rm.registerNode("h3:1236", 5122);
-    rm.waitForState(nm3.getNodeId(), NodeState.NEW);
-    rm.sendNodeStarted(nm3);
-    rm.waitForState(nm3.getNodeId(), NodeState.RUNNING);
-    RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes()
-        .get(nm3.getNodeId());
+    getRunningRMNode("h1", 1234, 5120);
+    // h2 will be in NEW state
+    getNewRMNode("h2", 1235, 5121);
+
+    RMNode node3 = getRunningRMNode("h3", 1236, 5122);
+    NodeId nodeId3 = node3.getNodeID();
+
+    RMNode node = rm.getRMContext().getRMNodes().get(nodeId3);
     NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(false,
         "test health report", System.currentTimeMillis());
-    NodeStatus nodeStatus = NodeStatus.newInstance(nm3.getNodeId(), 1,
+    NodeStatus nodeStatus = NodeStatus.newInstance(nodeId3, 1,
       new ArrayList<ContainerStatus>(), null, nodeHealth, null, null, null);
-    node.handle(new RMNodeStatusEvent(nm3.getNodeId(), nodeStatus, null));
-    rm.waitForState(nm3.getNodeId(), NodeState.UNHEALTHY);
+    ((RMNodeImpl) node)
+        .handle(new RMNodeStatusEvent(nodeId3, nodeStatus, null));
+    rm.waitForState(nodeId3, NodeState.UNHEALTHY);
 
     ClientResponse response =
         r.path("ws").path("v1").path("cluster").path("nodes")
@@ -163,14 +165,38 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals("incorrect number of elements", 3, nodeArray.length());
   }
 
+  private RMNode getRunningRMNode(String host, int port, int memory) {
+    RMNodeImpl rmnode1 = getNewRMNode(host, port, memory);
+    sendStartedEvent(rmnode1);
+    return rmnode1;
+  }
+
+  private void sendStartedEvent(RMNode node) {
+    ((RMNodeImpl) node)
+        .handle(new RMNodeStartedEvent(node.getNodeID(), null, null));
+  }
+
+  private void sendLostEvent(RMNode node) {
+    ((RMNodeImpl) node)
+        .handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE));
+  }
+
+  private RMNodeImpl getNewRMNode(String host, int port, int memory) {
+    NodeId nodeId = NodeId.newInstance(host, port);
+    RMNodeImpl nodeImpl = new RMNodeImpl(nodeId, rm.getRMContext(),
+        nodeId.getHost(), nodeId.getPort(), nodeId.getPort() + 1,
+        RackResolver.resolve(nodeId.getHost()), Resource.newInstance(memory, 4),
+        YarnVersionInfo.getVersion());
+    rm.getRMContext().getRMNodes().put(nodeId, nodeImpl);
+    return nodeImpl;
+  }
+
   @Test
   public void testNodesQueryNew() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.NEW);
+    getRunningRMNode("h1", 1234, 5120);
+    // h2 will be in NEW state
+    RMNode rmnode2 = getNewRMNode("h2", 1235, 5121);
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").queryParam("states", NodeState.NEW.toString())
@@ -185,14 +211,14 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals("incorrect number of elements", 1, nodeArray.length());
     JSONObject info = nodeArray.getJSONObject(0);
 
-    verifyNodeInfo(info, nm2);
+    verifyNodeInfo(info, rmnode2);
   }
 
   @Test
   public void testNodesQueryStateNone() throws JSONException, Exception {
     WebResource r = resource();
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+    getNewRMNode("h1", 1234, 5120);
+    getNewRMNode("h2", 1235, 5121);
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes")
@@ -207,8 +233,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
   @Test
   public void testNodesQueryStateInvalid() throws JSONException, Exception {
     WebResource r = resource();
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+    getNewRMNode("h1", 1234, 5120);
+    getNewRMNode("h2", 1235, 5121);
 
     try {
       r.path("ws").path("v1").path("cluster").path("nodes")
@@ -238,22 +264,16 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       WebServicesTestUtils.checkStringMatch("exception classname",
           "java.lang.IllegalArgumentException", classname);
 
-    } finally {
-      rm.stop();
     }
   }
   
   @Test
   public void testNodesQueryStateLost() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1234", 5120);
-    rm.sendNodeStarted(nm1);
-    rm.sendNodeStarted(nm2);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.RUNNING);
-    rm.sendNodeLost(nm1);
-    rm.sendNodeLost(nm2);
+    RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
+    sendLostEvent(rmnode1);
+    RMNode rmnode2 = getRunningRMNode("h2", 1235, 5121);
+    sendLostEvent(rmnode2);
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").queryParam("states", NodeState.LOST.toString())
@@ -280,14 +300,9 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
   @Test
   public void testSingleNodeQueryStateLost() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1234", 5120);
-    rm.sendNodeStarted(nm1);
-    rm.sendNodeStarted(nm2);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.RUNNING);
-    rm.sendNodeLost(nm1);
-    rm.sendNodeLost(nm2);
+    getRunningRMNode("h1", 1234, 5120);
+    RMNode rmnode2 = getRunningRMNode("h2", 1234, 5121);
+    sendLostEvent(rmnode2);
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").path("h2:1234").accept(MediaType.APPLICATION_JSON)
@@ -300,8 +315,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
 
     assertEquals("Incorrect Node Information.", "h2:1234", id);
 
-    NodeId nodeId = NodeId.newInstance("h2", 1234);
-    RMNode rmNode = rm.getRMContext().getInactiveRMNodes().get(nodeId);
+    RMNode rmNode =
+        rm.getRMContext().getInactiveRMNodes().get(rmnode2.getNodeID());
     WebServicesTestUtils.checkStringMatch("nodeHTTPAddress", "",
         info.getString("nodeHTTPAddress"));
     WebServicesTestUtils.checkStringMatch("state",
@@ -311,11 +326,9 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
   @Test
   public void testNodesQueryRunning() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.NEW);
+    getRunningRMNode("h1", 1234, 5120);
+    // h2 will be in NEW state
+    getNewRMNode("h2", 1235, 5121);
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").queryParam("states", "running")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
@@ -331,11 +344,9 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
   @Test
   public void testNodesQueryHealthyFalse() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.NEW);
+    getRunningRMNode("h1", 1234, 5120);
+    // h2 will be in NEW state
+    getNewRMNode("h2", 1235, 5121);
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").queryParam("states", "UNHEALTHY")
         .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
@@ -348,12 +359,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
   public void testNodesHelper(String path, String media) throws JSONException,
       Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.sendNodeStarted(nm2);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.RUNNING);
+    RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
+    RMNode rmnode2 = getRunningRMNode("h2", 1235, 5121);
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path(path).accept(media).get(ClientResponse.class);
@@ -368,36 +375,36 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     String id = info.get("id").toString();
 
     if (id.matches("h1:1234")) {
-      verifyNodeInfo(info, nm1);
-      verifyNodeInfo(nodeArray.getJSONObject(1), nm2);
+      verifyNodeInfo(info, rmnode1);
+      verifyNodeInfo(nodeArray.getJSONObject(1), rmnode2);
     } else {
-      verifyNodeInfo(info, nm2);
-      verifyNodeInfo(nodeArray.getJSONObject(1), nm1);
+      verifyNodeInfo(info, rmnode2);
+      verifyNodeInfo(nodeArray.getJSONObject(1), rmnode1);
     }
   }
 
   @Test
   public void testSingleNode() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    testSingleNodeHelper("h2:1235", nm2, MediaType.APPLICATION_JSON);
+    getRunningRMNode("h1", 1234, 5120);
+    RMNode rmnode2 = getRunningRMNode("h2", 1235, 5121);
+    testSingleNodeHelper("h2:1235", rmnode2, MediaType.APPLICATION_JSON);
   }
 
   @Test
   public void testSingleNodeSlash() throws JSONException, Exception {
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
-    testSingleNodeHelper("h1:1234/", nm1, MediaType.APPLICATION_JSON);
+    RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
+    getRunningRMNode("h2", 1235, 5121);
+    testSingleNodeHelper("h1:1234/", rmnode1, MediaType.APPLICATION_JSON);
   }
 
   @Test
   public void testSingleNodeDefault() throws JSONException, Exception {
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
-    testSingleNodeHelper("h1:1234/", nm1, "");
+    RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
+    getRunningRMNode("h2", 1235, 5121);
+    testSingleNodeHelper("h1:1234/", rmnode1, "");
   }
 
-  public void testSingleNodeHelper(String nodeid, MockNM nm, String media)
+  public void testSingleNodeHelper(String nodeid, RMNode nm, String media)
       throws JSONException, Exception {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("cluster")
@@ -412,8 +419,10 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
 
   @Test
   public void testNonexistNode() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+    // add h1 node in NEW state
+    getNewRMNode("h1", 1234, 5120);
+    // add h2 node in NEW state
+    getNewRMNode("h2", 1235, 5121);
     WebResource r = resource();
     try {
       r.path("ws").path("v1").path("cluster").path("nodes")
@@ -433,16 +442,14 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       String classname = exception.getString("javaClassName");
       verifyNonexistNodeException(message, type, classname);
 
-    } finally {
-      rm.stop();
     }
   }
 
   // test that the exception output defaults to JSON
   @Test
   public void testNonexistNodeDefault() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+    getNewRMNode("h1", 1234, 5120);
+    getNewRMNode("h2", 1235, 5121);
     WebResource r = resource();
     try {
       r.path("ws").path("v1").path("cluster").path("nodes")
@@ -460,16 +467,14 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       String type = exception.getString("exception");
       String classname = exception.getString("javaClassName");
       verifyNonexistNodeException(message, type, classname);
-    } finally {
-      rm.stop();
     }
   }
 
   // test that the exception output works in XML
   @Test
   public void testNonexistNodeXML() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+    getNewRMNode("h1", 1234, 5120);
+    getNewRMNode("h2", 1235, 5121);
     WebResource r = resource();
     try {
       r.path("ws").path("v1").path("cluster").path("nodes")
@@ -495,8 +500,6 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       String classname = WebServicesTestUtils.getXmlString(element,
           "javaClassName");
       verifyNonexistNodeException(message, type, classname);
-    } finally {
-      rm.stop();
     }
   }
 
@@ -511,8 +514,8 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
 
   @Test
   public void testInvalidNode() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+    getNewRMNode("h1", 1234, 5120);
+    getNewRMNode("h2", 1235, 5121);
 
     WebResource r = resource();
     try {
@@ -538,16 +541,13 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
           "IllegalArgumentException", type);
       WebServicesTestUtils.checkStringMatch("exception classname",
           "java.lang.IllegalArgumentException", classname);
-    } finally {
-      rm.stop();
     }
   }
 
   @Test
   public void testNodesXML() throws JSONException, Exception {
-    rm.start();
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
+    RMNodeImpl rmnode1 = getNewRMNode("h1", 1234, 5120);
     // MockNM nm2 = rm.registerNode("h2:1235", 5121);
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").accept(MediaType.APPLICATION_XML)
@@ -563,15 +563,14 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals("incorrect number of elements", 1, nodesApps.getLength());
     NodeList nodes = dom.getElementsByTagName("node");
     assertEquals("incorrect number of elements", 1, nodes.getLength());
-    verifyNodesXML(nodes, nm1);
-    rm.stop();
+    verifyNodesXML(nodes, rmnode1);
   }
 
   @Test
   public void testSingleNodesXML() throws JSONException, Exception {
-    rm.start();
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
+    // add h2 node in NEW state
+    RMNodeImpl rmnode1 = getNewRMNode("h1", 1234, 5120);
     // MockNM nm2 = rm.registerNode("h2:1235", 5121);
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").path("h1:1234").accept(MediaType.APPLICATION_XML)
@@ -587,16 +586,14 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     Document dom = db.parse(is);
     NodeList nodes = dom.getElementsByTagName("node");
     assertEquals("incorrect number of elements", 1, nodes.getLength());
-    verifyNodesXML(nodes, nm1);
-    rm.stop();
+    verifyNodesXML(nodes, rmnode1);
   }
 
   @Test
   public void testNodes2XML() throws JSONException, Exception {
-    rm.start();
     WebResource r = resource();
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+    getNewRMNode("h1", 1234, 5120);
+    getNewRMNode("h2", 1235, 5121);
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").accept(MediaType.APPLICATION_XML)
         .get(ClientResponse.class);
@@ -612,20 +609,17 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals("incorrect number of elements", 1, nodesApps.getLength());
     NodeList nodes = dom.getElementsByTagName("node");
     assertEquals("incorrect number of elements", 2, nodes.getLength());
-    rm.stop();
   }
   
   @Test
   public void testQueryAll() throws Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    MockNM nm3 = rm.registerNode("h3:1236", 5122);
-    rm.sendNodeStarted(nm1);
-    rm.sendNodeStarted(nm3);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
-    rm.waitForState(nm2.getNodeId(), NodeState.NEW);
-    rm.sendNodeLost(nm3);
+    getRunningRMNode("h1", 1234, 5120);
+    // add h2 node in NEW state
+    getNewRMNode("h2", 1235, 5121);
+    // add lost node
+    RMNode nm3 = getRunningRMNode("h3", 1236, 5122);
+    sendLostEvent(nm3);
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes")
@@ -643,23 +637,22 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
   @Test
   public void testNodesResourceUtilization() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    rm.sendNodeStarted(nm1);
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
+    RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
+    NodeId nodeId1 = rmnode1.getNodeID();
 
-    RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes()
-        .get(nm1.getNodeId());
+    RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes().get(nodeId1);
     NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true,
         "test health report", System.currentTimeMillis());
     ResourceUtilization nodeResource = ResourceUtilization.newInstance(4096, 0,
         (float) 10.5);
     ResourceUtilization containerResource = ResourceUtilization.newInstance(
         2048, 0, (float) 5.05);
-    NodeStatus nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0,
+    NodeStatus nodeStatus =
+        NodeStatus.newInstance(nodeId1, 0,
         new ArrayList<ContainerStatus>(), null, nodeHealth, containerResource,
         nodeResource, null);
-    node.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus, null));
-    rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
+    node.handle(new RMNodeStatusEvent(nodeId1, nodeStatus, null));
+    rm.waitForState(nodeId1, NodeState.RUNNING);
 
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("nodes").accept(MediaType.APPLICATION_JSON)
@@ -675,10 +668,11 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     JSONObject info = nodeArray.getJSONObject(0);
 
     // verify the resource utilization
-    verifyNodeInfo(info, nm1);
+    verifyNodeInfo(info, rmnode1);
   }
 
-  public void verifyNodesXML(NodeList nodes, MockNM nm) throws JSONException,
+  public void verifyNodesXML(NodeList nodes, RMNode nm)
+      throws JSONException,
       Exception {
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
@@ -707,7 +701,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     }
   }
 
-  public void verifyNodeInfo(JSONObject nodeInfo, MockNM nm)
+  public void verifyNodeInfo(JSONObject nodeInfo, RMNode nm)
       throws JSONException, Exception {
     assertEquals("incorrect number of elements", 14, nodeInfo.length());
 
@@ -729,7 +723,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         resourceInfo.getDouble("containersCPUUsage"));
   }
 
-  public void verifyNodeInfoGeneric(MockNM nm, String state, String rack,
+  public void verifyNodeInfoGeneric(RMNode node, String state, String rack,
       String id, String nodeHostName,
       String nodeHTTPAddress, long lastHealthUpdate, String healthReport,
       int numContainers, long usedMemoryMB, long availMemoryMB, long usedVirtualCores, 
@@ -739,20 +733,20 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
       double containersCPUUsage)
       throws JSONException, Exception {
 
-    RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
     ResourceScheduler sched = rm.getResourceScheduler();
-    SchedulerNodeReport report = sched.getNodeReport(nm.getNodeId());
+    SchedulerNodeReport report = sched.getNodeReport(node.getNodeID());
 
     WebServicesTestUtils.checkStringMatch("state", node.getState().toString(),
         state);
     WebServicesTestUtils.checkStringMatch("rack", node.getRackName(), rack);
-    WebServicesTestUtils.checkStringMatch("id", nm.getNodeId().toString(), id);
-    WebServicesTestUtils.checkStringMatch("nodeHostName", nm.getNodeId()
-        .getHost(), nodeHostName);
+    WebServicesTestUtils.checkStringMatch("id", node.getNodeID().toString(),
+        id);
+    WebServicesTestUtils.checkStringMatch("nodeHostName",
+        node.getNodeID().getHost(), nodeHostName);
     WebServicesTestUtils.checkStringMatch("healthReport",
         String.valueOf(node.getHealthReport()), healthReport);
-    String expectedHttpAddress = nm.getNodeId().getHost() + ":"
-        + nm.getHttpPort();
+    String expectedHttpAddress =
+        node.getNodeID().getHost() + ":" + node.getHttpPort();
     WebServicesTestUtils.checkStringMatch("nodeHTTPAddress",
         expectedHttpAddress, nodeHTTPAddress);
     WebServicesTestUtils.checkStringMatch("version",

+ 16 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java

@@ -61,7 +61,7 @@ public class EntityCacheItem {
    * Set the application logs to this cache item. The entity group should be
    * associated with this application.
    *
-   * @param incomingAppLogs
+   * @param incomingAppLogs Application logs this cache item mapped to
    */
   public synchronized void setAppLogs(
       EntityGroupFSTimelineStore.AppLogs incomingAppLogs) {
@@ -80,18 +80,21 @@ public class EntityCacheItem {
    * rescan and then load new data. The refresh process is synchronized with
    * other operations on the same cache item.
    *
-   * @param groupId
-   * @param aclManager
-   * @param jsonFactory
-   * @param objMapper
+   * @param groupId Group id of the cache
+   * @param aclManager ACL manager for the timeline storage
+   * @param jsonFactory JSON factory for the storage
+   * @param objMapper Object mapper for the storage
+   * @param metrics Metrics to trace the status of the entity group store
    * @return a {@link org.apache.hadoop.yarn.server.timeline.TimelineStore}
    *         object filled with all entities in the group.
    * @throws IOException
    */
   public synchronized TimelineStore refreshCache(TimelineEntityGroupId groupId,
       TimelineACLsManager aclManager, JsonFactory jsonFactory,
-      ObjectMapper objMapper) throws IOException {
+      ObjectMapper objMapper, EntityGroupFSTimelineStoreMetrics metrics)
+      throws IOException {
     if (needRefresh()) {
+      long startTime = Time.monotonicNow();
       // If an application is not finished, we only update summary logs (and put
       // new entities into summary storage).
       // Otherwise, since the application is done, we can update detail logs.
@@ -106,9 +109,12 @@ public class EntityCacheItem {
               "LeveldbCache." + groupId);
           store.init(config);
           store.start();
+        } else {
+          // Store is not null, the refresh is triggered by stale storage.
+          metrics.incrCacheStaleRefreshes();
         }
         List<LogInfo> removeList = new ArrayList<>();
-        try(TimelineDataManager tdm =
+        try (TimelineDataManager tdm =
                 new TimelineDataManager(store, aclManager)) {
           tdm.init(config);
           tdm.start();
@@ -133,8 +139,10 @@ public class EntityCacheItem {
         appLogs.getDetailLogs().removeAll(removeList);
       }
       updateRefreshTimeToNow();
+      metrics.addCacheRefreshTime(Time.monotonicNow() - startTime);
     } else {
       LOG.debug("Cache new enough, skip refreshing");
+      metrics.incrNoRefreshCacheRead();
     }
     return store;
   }
@@ -142,7 +150,7 @@ public class EntityCacheItem {
   /**
    * Release the cache item for the given group id.
    *
-   * @param groupId
+   * @param groupId the group id that the cache should release
    */
   public synchronized void releaseCache(TimelineEntityGroupId groupId) {
     try {

+ 20 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java

@@ -128,12 +128,17 @@ public class EntityGroupFSTimelineStore extends CompositeService
   private List<TimelineEntityGroupPlugin> cacheIdPlugins;
   private Map<TimelineEntityGroupId, EntityCacheItem> cachedLogs;
 
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  EntityGroupFSTimelineStoreMetrics metrics;
+
   public EntityGroupFSTimelineStore() {
     super(EntityGroupFSTimelineStore.class.getSimpleName());
   }
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
+    metrics = EntityGroupFSTimelineStoreMetrics.create();
     summaryStore = createSummaryStore();
     addService(summaryStore);
 
@@ -171,6 +176,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
               if (cacheItem.getAppLogs().isDone()) {
                 appIdLogMap.remove(groupId.getApplicationId());
               }
+              metrics.incrCacheEvicts();
               return true;
             }
             return false;
@@ -316,6 +322,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
   @InterfaceAudience.Private
   @VisibleForTesting
   int scanActiveLogs() throws IOException {
+    long startTime = Time.monotonicNow();
     RemoteIterator<FileStatus> iter = list(activeRootPath);
     int logsToScanCount = 0;
     while (iter.hasNext()) {
@@ -331,6 +338,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
         LOG.debug("Unable to parse entry {}", name);
       }
     }
+    metrics.addActiveLogDirScanTime(Time.monotonicNow() - startTime);
     return logsToScanCount;
   }
 
@@ -423,6 +431,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
               if (!fs.delete(dirpath, true)) {
                 LOG.error("Unable to remove " + dirpath);
               }
+              metrics.incrLogsDirsCleaned();
             } catch (IOException e) {
               LOG.error("Unable to remove " + dirpath, e);
             }
@@ -588,6 +597,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
     @VisibleForTesting
     synchronized void parseSummaryLogs(TimelineDataManager tdm)
         throws IOException {
+      long startTime = Time.monotonicNow();
       if (!isDone()) {
         LOG.debug("Try to parse summary log for log {} in {}",
             appId, appDirPath);
@@ -605,8 +615,10 @@ public class EntityGroupFSTimelineStore extends CompositeService
       List<LogInfo> removeList = new ArrayList<LogInfo>();
       for (LogInfo log : summaryLogs) {
         if (fs.exists(log.getPath(appDirPath))) {
-          log.parseForStore(tdm, appDirPath, isDone(), jsonFactory,
+          long summaryEntityParsed
+              = log.parseForStore(tdm, appDirPath, isDone(), jsonFactory,
               objMapper, fs);
+          metrics.incrEntitiesReadToSummary(summaryEntityParsed);
         } else {
           // The log may have been removed, remove the log
           removeList.add(log);
@@ -615,6 +627,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
         }
       }
       summaryLogs.removeAll(removeList);
+      metrics.addSummaryLogReadTime(Time.monotonicNow() - startTime);
     }
 
     // scans for new logs and returns the modification timestamp of the
@@ -787,6 +800,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
     @Override
     public void run() {
       LOG.debug("Cleaner starting");
+      long startTime = Time.monotonicNow();
       try {
         cleanLogs(doneRootPath, fs, logRetainMillis);
       } catch (Exception e) {
@@ -796,6 +810,8 @@ public class EntityGroupFSTimelineStore extends CompositeService
         } else {
           LOG.error("Error cleaning files", e);
         }
+      } finally {
+        metrics.addLogCleanTime(Time.monotonicNow() - startTime);
       }
       LOG.debug("Cleaner finished");
     }
@@ -824,11 +840,13 @@ public class EntityGroupFSTimelineStore extends CompositeService
       if (storeForId != null) {
         LOG.debug("Adding {} as a store for the query", storeForId.getName());
         stores.add(storeForId);
+        metrics.incrGetEntityToDetailOps();
       }
     }
     if (stores.size() == 0) {
       LOG.debug("Using summary store for {}", entityType);
       stores.add(this.summaryStore);
+      metrics.incrGetEntityToSummaryOps();
     }
     return stores;
   }
@@ -898,7 +916,7 @@ public class EntityGroupFSTimelineStore extends CompositeService
       AppLogs appLogs = cacheItem.getAppLogs();
       LOG.debug("try refresh cache {} {}", groupId, appLogs.getAppId());
       store = cacheItem.refreshCache(groupId, aclManager, jsonFactory,
-          objMapper);
+          objMapper, metrics);
     } else {
       LOG.warn("AppLogs for group id {} is null", groupId);
     }

+ 160 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStoreMetrics.java

@@ -0,0 +1,160 @@
+/*
+ * 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.timeline;
+
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableStat;
+
+/**
+ * This class tracks metrics for the EntityGroupFSTimelineStore. It tracks
+ * the read and write metrics for timeline server v1.5. It serves as a
+ * complement to {@link TimelineDataManagerMetrics}.
+ */
+@Metrics(about="Metrics for EntityGroupFSTimelineStore", context="yarn")
+public class EntityGroupFSTimelineStoreMetrics {
+  private static final String DEFAULT_VALUE_WITH_SCALE = "TimeMs";
+
+  // General read related metrics
+  @Metric("getEntity calls to summary storage")
+  private MutableCounterLong getEntityToSummaryOps;
+
+  @Metric("getEntity calls to detail storage")
+  private MutableCounterLong getEntityToDetailOps;
+
+  // Summary data related metrics
+  @Metric(value = "summary log read ops and time",
+      valueName = DEFAULT_VALUE_WITH_SCALE)
+  private MutableStat summaryLogRead;
+
+  @Metric("entities read into the summary storage")
+  private MutableCounterLong entitiesReadToSummary;
+
+  // Detail data cache related metrics
+  @Metric("cache storage read that does not require a refresh")
+  private MutableCounterLong noRefreshCacheRead;
+
+  @Metric("cache storage refresh due to the cached storage is stale")
+  private MutableCounterLong cacheStaleRefreshes;
+
+  @Metric("cache storage evicts")
+  private MutableCounterLong cacheEvicts;
+
+  @Metric(value = "cache storage refresh ops and time",
+      valueName = DEFAULT_VALUE_WITH_SCALE)
+  private MutableStat cacheRefresh;
+
+  // Log scanner and cleaner related metrics
+  @Metric(value = "active log scan ops and time",
+      valueName = DEFAULT_VALUE_WITH_SCALE)
+  private MutableStat activeLogDirScan;
+
+  @Metric(value = "log cleaner purging ops and time",
+      valueName = DEFAULT_VALUE_WITH_SCALE)
+  private MutableStat logClean;
+
+  @Metric("log cleaner dirs purged")
+  private MutableCounterLong logsDirsCleaned;
+
+  private static EntityGroupFSTimelineStoreMetrics instance = null;
+
+  EntityGroupFSTimelineStoreMetrics() {
+  }
+
+  public static synchronized EntityGroupFSTimelineStoreMetrics create() {
+    if (instance == null) {
+      MetricsSystem ms = DefaultMetricsSystem.instance();
+      instance = ms.register(new EntityGroupFSTimelineStoreMetrics());
+    }
+    return instance;
+  }
+
+  // Setters
+  // General read related
+  public void incrGetEntityToSummaryOps() {
+    getEntityToSummaryOps.incr();
+  }
+
+  public void incrGetEntityToDetailOps() {
+    getEntityToDetailOps.incr();
+  }
+
+  // Summary data related
+  public void addSummaryLogReadTime(long msec) {
+    summaryLogRead.add(msec);
+  }
+
+  public void incrEntitiesReadToSummary(long delta) {
+    entitiesReadToSummary.incr(delta);
+  }
+
+  // Cache related
+  public void incrNoRefreshCacheRead() {
+    noRefreshCacheRead.incr();
+  }
+
+  public void incrCacheStaleRefreshes() {
+    cacheStaleRefreshes.incr();
+  }
+
+  public void incrCacheEvicts() {
+    cacheEvicts.incr();
+  }
+
+  public void addCacheRefreshTime(long msec) {
+    cacheRefresh.add(msec);
+  }
+
+  // Log scanner and cleaner related
+  public void addActiveLogDirScanTime(long msec) {
+    activeLogDirScan.add(msec);
+  }
+
+  public void addLogCleanTime(long msec) {
+    logClean.add(msec);
+  }
+
+  public void incrLogsDirsCleaned() {
+    logsDirsCleaned.incr();
+  }
+
+  // Getters
+  MutableCounterLong getEntitiesReadToSummary() {
+    return entitiesReadToSummary;
+  }
+
+  MutableCounterLong getLogsDirsCleaned() {
+    return logsDirsCleaned;
+  }
+
+  MutableCounterLong getGetEntityToSummaryOps() {
+    return getEntityToSummaryOps;
+  }
+
+  MutableCounterLong getGetEntityToDetailOps() {
+    return getEntityToDetailOps;
+  }
+
+  MutableStat getCacheRefresh() {
+    return cacheRefresh;
+  }
+}
+

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LogInfo.java

@@ -98,13 +98,14 @@ abstract class LogInfo {
     ));
   }
 
-  public void parseForStore(TimelineDataManager tdm, Path appDirPath,
+  public long parseForStore(TimelineDataManager tdm, Path appDirPath,
       boolean appCompleted, JsonFactory jsonFactory, ObjectMapper objMapper,
       FileSystem fs) throws IOException {
     LOG.debug("Parsing for log dir {} on attempt {}", appDirPath,
         attemptDirName);
     Path logPath = getPath(appDirPath);
     FileStatus status = fs.getFileStatus(logPath);
+    long numParsed = 0;
     if (status != null) {
       long startTime = Time.monotonicNow();
       try {
@@ -113,6 +114,7 @@ abstract class LogInfo {
             objMapper, fs);
         LOG.info("Parsed {} entities from {} in {} msec",
             count, logPath, Time.monotonicNow() - startTime);
+        numParsed += count;
       } catch (RuntimeException e) {
         // If AppLogs cannot parse this log, it may be corrupted or just empty
         if (e.getCause() instanceof JsonParseException &&
@@ -125,6 +127,7 @@ abstract class LogInfo {
     } else {
       LOG.warn("{} no longer exists. Skip for scanning. ", logPath);
     }
+    return numParsed;
   }
 
   private long parsePath(TimelineDataManager tdm, Path logPath,

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableStat;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -177,12 +179,15 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
   @Test
   public void testParseSummaryLogs() throws Exception {
     TimelineDataManager tdm = PluginStoreTestUtils.getTdmWithMemStore(config);
+    MutableCounterLong scanned = store.metrics.getEntitiesReadToSummary();
+    long beforeScan = scanned.value();
     EntityGroupFSTimelineStore.AppLogs appLogs =
         store.new AppLogs(TEST_APPLICATION_ID, testAppDirPath,
         AppState.COMPLETED);
     appLogs.scanForLogs();
     appLogs.parseSummaryLogs(tdm);
     PluginStoreTestUtils.verifyTestEntities(tdm);
+    assertEquals(beforeScan + 2L, scanned.value());
   }
 
   @Test
@@ -227,6 +232,8 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     fs.mkdirs(dirPathEmpty);
 
     // Should retain all logs after this run
+    MutableCounterLong dirsCleaned = store.metrics.getLogsDirsCleaned();
+    long before = dirsCleaned.value();
     store.cleanLogs(testDoneDirPath, fs, 10000);
     assertTrue(fs.exists(irrelevantDirPath));
     assertTrue(fs.exists(irrelevantFilePath));
@@ -256,6 +263,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     // appDirClean and appDirEmpty should be cleaned up
     assertFalse(fs.exists(appDirClean));
     assertFalse(fs.exists(appDirEmpty));
+    assertEquals(before + 2L, dirsCleaned.value());
   }
 
   @Test
@@ -272,6 +280,12 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     cacheItem.setAppLogs(appLogs);
     store.setCachedLogs(
         EntityGroupPlugInForTest.getStandardTimelineGroupId(), cacheItem);
+    MutableCounterLong detailLogEntityRead =
+        store.metrics.getGetEntityToDetailOps();
+    MutableStat cacheRefresh = store.metrics.getCacheRefresh();
+    long numEntityReadBefore = detailLogEntityRead.value();
+    long cacheRefreshBefore = cacheRefresh.lastStat().numSamples();
+
     // Generate TDM
     TimelineDataManager tdm
         = PluginStoreTestUtils.getTdmWithStore(config, store);
@@ -290,6 +304,9 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     for (TimelineEntity entity : entities.getEntities()) {
       assertEquals(entityNew.getStartTime(), entity.getStartTime());
     }
+    // Verify metrics
+    assertEquals(numEntityReadBefore + 2L, detailLogEntityRead.value());
+    assertEquals(cacheRefreshBefore + 1L, cacheRefresh.lastStat().numSamples());
   }
 
   @Test
@@ -298,6 +315,9 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     EntityGroupFSTimelineStore.AppLogs appLogs =
         store.new AppLogs(TEST_APPLICATION_ID, testAppDirPath,
         AppState.COMPLETED);
+    MutableCounterLong summaryLogEntityRead
+        = store.metrics.getGetEntityToSummaryOps();
+    long numEntityReadBefore = summaryLogEntityRead.value();
     TimelineDataManager tdm
         = PluginStoreTestUtils.getTdmWithStore(config, store);
     appLogs.scanForLogs();
@@ -313,6 +333,8 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     for (TimelineEntity entity : entities.getEntities()) {
       assertEquals((Long) 123l, entity.getStartTime());
     }
+    // Verify metrics
+    assertEquals(numEntityReadBefore + 5L, summaryLogEntityRead.value());
 
   }