瀏覽代碼

HADOOP-17524. Remove EventCounter and Log counters from JVM Metrics (#2909)

Reviewed-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
Viraj Jasani 4 年之前
父節點
當前提交
9179638017

+ 0 - 34
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/EventCounter.java

@@ -1,34 +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.log;
-
-/**
- * A log4J Appender that simply counts logging events in three levels:
- * fatal, error and warn. The class name is used in log4j.properties
- * @deprecated use {@link org.apache.hadoop.log.metrics.EventCounter} 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 - 100
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/metrics/EventCounter.java

@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.log.metrics;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import org.apache.log4j.AppenderSkeleton;
-import org.apache.log4j.Level;
-import org.apache.log4j.spi.LoggingEvent;
-
-/**
- * A log4J Appender that simply counts logging events in three levels:
- * fatal, error and warn. The class name is used in log4j.properties
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class EventCounter extends AppenderSkeleton {
-  private static final int FATAL = 0;
-  private static final int ERROR = 1;
-  private static final int WARN = 2;
-  private static final int INFO = 3;
-
-  private static class EventCounts {
-    private final long[] counts = {0, 0, 0, 0};
-
-    private synchronized void incr(int i) {
-      ++counts[i];
-    }
-
-    private synchronized long get(int i) {
-      return counts[i];
-    }
-  }
-
-  private static EventCounts counts = new EventCounts();
-
-  @InterfaceAudience.Private
-  public static long getFatal() {
-    return counts.get(FATAL);
-  }
-
-  @InterfaceAudience.Private
-  public static long getError() {
-    return counts.get(ERROR);
-  }
-
-  @InterfaceAudience.Private
-  public static long getWarn() {
-    return counts.get(WARN);
-  }
-
-  @InterfaceAudience.Private
-  public static long getInfo() {
-    return counts.get(INFO);
-  }
-
-  @Override
-  public void append(LoggingEvent event) {
-    Level level = event.getLevel();
-    // depends on the api, == might not work
-    // see HADOOP-7055 for details
-    if (level.equals(Level.INFO)) {
-      counts.incr(INFO);
-    }
-    else if (level.equals(Level.WARN)) {
-      counts.incr(WARN);
-    }
-    else if (level.equals(Level.ERROR)) {
-      counts.incr(ERROR);
-    }
-    else if (level.equals(Level.FATAL)) {
-      counts.incr(FATAL);
-    }
-  }
-
-  @Override
-  public void close() {
-  }
-
-  @Override
-  public boolean requiresLayout() {
-    return false;
-  }
-}

+ 0 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.log.metrics.EventCounter;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -154,7 +153,6 @@ public class JvmMetrics implements MetricsSource {
     } else {
       getThreadUsageFromGroup(rb);
     }
-    getEventCounters(rb);
   }
 
   private void getMemoryUsage(MetricsRecordBuilder rb) {
@@ -284,10 +282,4 @@ public class JvmMetrics implements MetricsSource {
         .addGauge(ThreadsTerminated, threadsTerminated);
   }
 
-  private void getEventCounters(MetricsRecordBuilder rb) {
-    rb.addCounter(LogFatal, EventCounter.getFatal())
-      .addCounter(LogError, EventCounter.getError())
-      .addCounter(LogWarn, EventCounter.getWarn())
-      .addCounter(LogInfo, EventCounter.getInfo());
-  }
 }

+ 0 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java

@@ -78,13 +78,8 @@ public class TestJvmMetrics {
     for (JvmMetricsInfo info : JvmMetricsInfo.values()) {
       if (info.name().startsWith("Mem")) {
         verify(rb).addGauge(eq(info), anyFloat());
-      } else if (info.name().startsWith("Gc") &&
-          !info.name().equals("GcTimePercentage")) {
-        verify(rb).addCounter(eq(info), anyLong());
       } else if (info.name().startsWith("Threads")) {
         verify(rb).addGauge(eq(info), anyInt());
-      } else if (info.name().startsWith("Log")) {
-        verify(rb).addCounter(eq(info), anyLong());
       }
     }
   }