浏览代码

YARN-6958. Moving logging APIs over to slf4j in hadoop-yarn-server-timelineservice. Contributed by Yeliang Cang.

Akira Ajisaka 7 年之前
父节点
当前提交
63cfcb90ac
共有 25 个文件被更改,包括 102 次插入91 次删除
  1. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
  2. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
  3. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
  4. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
  5. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
  6. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
  7. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
  8. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
  9. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
  10. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
  11. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
  12. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
  13. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
  14. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
  15. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
  16. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
  17. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
  18. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
  19. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
  20. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
  21. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
  22. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
  23. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
  24. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
  25. 0 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java

@@ -19,8 +19,6 @@
 package org.apache.hadoop.yarn.server.timeline;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +32,8 @@ import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DBIterator;
 import org.iq80.leveldb.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -58,8 +58,8 @@ import java.util.Map;
 @Private
 @Unstable
 public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
-  private static final Log LOG
-      = LogFactory.getLog(LevelDBCacheTimelineStore.class);
+  private static final Logger LOG
+      = LoggerFactory.getLogger(LevelDBCacheTimelineStore.class);
   private static final String CACHED_LDB_FILE_PREFIX = "-timeline-cache.ldb";
   private String dbId;
   private DB entityDb;
@@ -102,7 +102,7 @@ public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
         localFS.setPermission(dbPath, LeveldbUtils.LEVELDB_DIR_UMASK);
       }
     } finally {
-      IOUtils.cleanup(LOG, localFS);
+      IOUtils.cleanupWithLogger(LOG, localFS);
     }
     LOG.info("Using leveldb path " + dbPath);
     entityDb = factory.open(new File(dbPath.toString()), options);
@@ -113,7 +113,7 @@ public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
 
   @Override
   protected synchronized void serviceStop() throws Exception {
-    IOUtils.cleanup(LOG, entityDb);
+    IOUtils.cleanupWithLogger(LOG, entityDb);
     Path dbPath = new Path(
         configuration.get(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH),
         dbId + CACHED_LDB_FILE_PREFIX);
@@ -125,7 +125,7 @@ public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
               "timeline store " + dbPath);
       }
     } finally {
-      IOUtils.cleanup(LOG, localFS);
+      IOUtils.cleanupWithLogger(LOG, localFS);
     }
     super.serviceStop();
   }

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java

@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
 import org.apache.hadoop.hbase.filter.FamilyFilter;
@@ -36,13 +34,16 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Set of utility methods used by timeline filter classes.
  */
 public final class TimelineFilterUtils {
 
-  private static final Log LOG = LogFactory.getLog(TimelineFilterUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineFilterUtils.class);
 
   private TimelineFilterUtils() {
   }

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java

@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Connection;
@@ -34,6 +32,8 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilter
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
 import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader;
 import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * HBase based implementation for {@link TimelineReader}.
@@ -41,8 +41,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEnti
 public class HBaseTimelineReaderImpl
     extends AbstractService implements TimelineReader {
 
-  private static final Log LOG = LogFactory
-      .getLog(HBaseTimelineReaderImpl.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(HBaseTimelineReaderImpl.class);
 
   private Configuration hbaseConf = null;
   private Connection conn;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java

@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Set;
 
-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.conf.Configuration;
@@ -65,6 +63,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This implements a hbase based backend for storing the timeline entity
@@ -76,8 +76,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 public class HBaseTimelineWriterImpl extends AbstractService implements
     TimelineWriter {
 
-  private static final Log LOG = LogFactory
-      .getLog(HBaseTimelineWriterImpl.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(HBaseTimelineWriterImpl.class);
 
   private Connection conn;
   private TypedBufferedMutator<EntityTable> entityTable;

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java

@@ -29,8 +29,6 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
 import org.apache.commons.lang.StringUtils;
-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.conf.Configuration;
@@ -46,6 +44,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTa
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This creates the schema for a hbase based backend for storing application
@@ -58,7 +58,8 @@ public final class TimelineSchemaCreator {
   }
 
   final static String NAME = TimelineSchemaCreator.class.getSimpleName();
-  private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineSchemaCreator.class);
   private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s";
   private static final String APP_METRICS_TTL_OPTION_SHORT = "ma";
   private static final String APP_TABLE_NAME_SHORT = "a";

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.application;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The application table as column families info, config and metrics. Info
@@ -99,7 +99,8 @@ public class ApplicationTable extends BaseTable<ApplicationTable> {
   /** default max number of versions. */
   private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000;
 
-  private static final Log LOG = LogFactory.getLog(ApplicationTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ApplicationTable.class);
 
   public ApplicationTable() {
     super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java

@@ -18,8 +18,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
 
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -29,6 +27,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
@@ -68,7 +68,8 @@ public class AppToFlowTable extends BaseTable<AppToFlowTable> {
   /** default value for app_flow table name. */
   private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow";
 
-  private static final Log LOG = LogFactory.getLog(AppToFlowTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AppToFlowTable.class);
 
   public AppToFlowTable() {
     super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java

@@ -24,13 +24,14 @@ import java.util.Map.Entry;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * This class is meant to be used only by explicit Columns, and not directly to
  * write by clients.
@@ -38,7 +39,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
  * @param <T> refers to the table.
  */
 public class ColumnHelper<T> {
-  private static final Log LOG = LogFactory.getLog(ColumnHelper.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ColumnHelper.class);
 
   private final ColumnFamily<T> columnFamily;
 

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java

@@ -17,8 +17,6 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -30,6 +28,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCom
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.List;
@@ -41,8 +41,8 @@ import java.util.Map;
 public final class HBaseTimelineStorageUtils {
   /** milliseconds in one day. */
   public static final long MILLIS_ONE_DAY = 86400000L;
-  private static final Log LOG =
-      LogFactory.getLog(HBaseTimelineStorageUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(HBaseTimelineStorageUtils.class);
 
   private HBaseTimelineStorageUtils() {
   }

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The entity table as column families info, config and metrics. Info stores
@@ -99,7 +99,8 @@ public class EntityTable extends BaseTable<EntityTable> {
   /** default max number of versions. */
   private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000;
 
-  private static final Log LOG = LogFactory.getLog(EntityTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(EntityTable.class);
 
   public EntityTable() {
     super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -29,6 +27,8 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The flow activity table has column family info
@@ -63,7 +63,8 @@ public class FlowActivityTable extends BaseTable<FlowActivityTable> {
   public static final String DEFAULT_TABLE_NAME =
       "timelineservice.flowactivity";
 
-  private static final Log LOG = LogFactory.getLog(FlowActivityTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowActivityTable.class);
 
   /** default max number of versions. */
   public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java

@@ -24,8 +24,6 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -50,13 +48,16 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Coprocessor for flow run table.
  */
 public class FlowRunCoprocessor extends BaseRegionObserver {
 
-  private static final Log LOG = LogFactory.getLog(FlowRunCoprocessor.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowRunCoprocessor.class);
   private boolean isFlowRunRegion = false;
 
   private Region region;

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -29,6 +27,8 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The flow run table has column family info
@@ -94,7 +94,8 @@ public class FlowRunTable extends BaseTable<FlowRunTable> {
   /** default value for flowrun table name. */
   public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun";
 
-  private static final Log LOG = LogFactory.getLog(FlowRunTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowRunTable.class);
 
   /** default max number of versions. */
   public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java

@@ -27,8 +27,6 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -52,6 +50,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGen
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Invoked via the coprocessor when a Get or a Scan is issued for flow run
@@ -62,7 +62,8 @@ import com.google.common.annotations.VisibleForTesting;
  */
 class FlowScanner implements RegionScanner, Closeable {
 
-  private static final Log LOG = LogFactory.getLog(FlowScanner.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowScanner.class);
 
   /**
    * use a special application id to represent the flow id this is needed since

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java

@@ -27,8 +27,6 @@ import java.util.NavigableSet;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
@@ -54,6 +52,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The base class for reading and deserializing timeline entities from the
@@ -61,7 +61,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn
  * entities that are being requested.
  */
 public abstract class TimelineEntityReader {
-  private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineEntityReader.class);
 
   private final boolean singleEntityRead;
   private TimelineReaderContext context;

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java

@@ -19,8 +19,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -32,6 +30,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.HashSet;
 import java.util.Map;
@@ -48,7 +48,8 @@ import java.util.concurrent.TimeUnit;
 @Private
 @Unstable
 public class AppLevelTimelineCollector extends TimelineCollector {
-  private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollector.class);
 
   private final static int AGGREGATION_EXECUTOR_NUM_THREADS = 1;
   private final static int AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS = 15;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java

@@ -26,8 +26,6 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.HashMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -47,6 +45,8 @@ import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class on the NodeManager side that manages adding and removing collectors and
@@ -55,8 +55,8 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 @Unstable
 public class NodeTimelineCollectorManager extends TimelineCollectorManager {
-  private static final Log LOG =
-      LogFactory.getLog(NodeTimelineCollectorManager.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NodeTimelineCollectorManager.class);
 
   // REST server for this collector manager.
   private HttpServer2 timelineRestServer;

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java

@@ -23,8 +23,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -43,6 +41,8 @@ import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The top-level server for the per-node timeline collector manager. Currently
@@ -52,8 +52,8 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 @Unstable
 public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
-  private static final Log LOG =
-      LogFactory.getLog(PerNodeTimelineCollectorsAuxService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PerNodeTimelineCollectorsAuxService.class);
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
 
   private final NodeTimelineCollectorManager collectorManager;
@@ -209,7 +209,7 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
       auxService.init(conf);
       auxService.start();
     } catch (Throwable t) {
-      LOG.fatal("Error starting PerNodeTimelineCollectorServer", t);
+      LOG.error("Error starting PerNodeTimelineCollectorServer", t);
       ExitUtil.terminate(-1, "Error starting PerNodeTimelineCollectorServer");
     }
     return auxService;

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java

@@ -26,8 +26,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -39,6 +37,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Service that handles writes to the timeline service and writes them to the
@@ -51,7 +51,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 @Unstable
 public abstract class TimelineCollector extends CompositeService {
 
-  private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollector.class);
   public static final String SEPARATOR = "_";
 
   private TimelineWriter writer;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java

@@ -26,8 +26,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-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.conf.Configuration;
@@ -39,6 +37,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class that manages adding and removing collectors and their lifecycle. It
@@ -48,8 +48,8 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class TimelineCollectorManager extends AbstractService {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineCollectorManager.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollectorManager.class);
 
   private TimelineWriter writer;
   private ScheduledExecutorService writerFlusher;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java

@@ -36,8 +36,6 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -58,6 +56,8 @@ import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.inject.Singleton;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The main per-node REST end point for timeline service writes. It is
@@ -69,8 +69,8 @@ import com.google.inject.Singleton;
 @Singleton
 @Path("/ws/v2/timeline")
 public class TimelineCollectorWebService {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineCollectorWebService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollectorWebService.class);
 
   private @Context ServletContext context;
 

+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java

@@ -25,8 +25,6 @@ import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -48,12 +46,15 @@ import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** Main class for Timeline Reader. */
 @Private
 @Unstable
 public class TimelineReaderServer extends CompositeService {
-  private static final Log LOG = LogFactory.getLog(TimelineReaderServer.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineReaderServer.class);
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
   static final String TIMELINE_READER_MANAGER_ATTR =
       "timeline.reader.manager";
@@ -203,7 +204,7 @@ public class TimelineReaderServer extends CompositeService {
       timelineReaderServer.init(conf);
       timelineReaderServer.start();
     } catch (Throwable t) {
-      LOG.fatal("Error starting TimelineReaderWebServer", t);
+      LOG.error("Error starting TimelineReaderWebServer", t);
       ExitUtil.terminate(-1, "Error starting TimelineReaderWebServer");
     }
     return timelineReaderServer;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java

@@ -40,8 +40,6 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.http.JettyUtils;
@@ -57,6 +55,8 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Singleton;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** REST end point for Timeline Reader. */
 @Private
@@ -64,8 +64,8 @@ import com.google.inject.Singleton;
 @Singleton
 @Path("/ws/v2/timeline")
 public class TimelineReaderWebServices {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineReaderWebServices.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineReaderWebServices.class);
 
   @Context private ServletContext ctxt;
 

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java

@@ -39,8 +39,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVParser;
 import org.apache.commons.csv.CSVRecord;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@@ -54,6 +52,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStor
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *  File System based implementation for TimelineReader. This implementation may
@@ -64,8 +64,8 @@ import com.google.common.annotations.VisibleForTesting;
 public class FileSystemTimelineReaderImpl extends AbstractService
     implements TimelineReader {
 
-  private static final Log LOG =
-      LogFactory.getLog(FileSystemTimelineReaderImpl.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FileSystemTimelineReaderImpl.class);
 
   private String rootPath;
   private static final String ENTITIES_DIR = "entities";

+ 0 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java

@@ -23,8 +23,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@@ -48,8 +46,6 @@ public final class TimelineStorageUtils {
   private TimelineStorageUtils() {
   }
 
-  private static final Log LOG = LogFactory.getLog(TimelineStorageUtils.class);
-
   /**
    * Matches key-values filter. Used for relatesTo/isRelatedTo filters.
    *