瀏覽代碼

Revert "YARN-6873. Moving logging APIs over to slf4j in hadoop-yarn-server-applicationhistoryservice. Contributed by Yeliang Cang."

This reverts commit 1a78c0ff016097930edf68e8278f826b637e918c.
Akira Ajisaka 7 年之前
父節點
當前提交
a4aa1cb405
共有 18 個文件被更改,包括 107 次插入111 次删除
  1. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
  2. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
  3. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
  4. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
  5. 11 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
  6. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
  7. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
  8. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
  9. 18 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
  10. 7 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDB.java
  11. 11 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java
  12. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
  13. 15 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/recovery/LeveldbTimelineStateStore.java
  14. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java
  15. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
  16. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
  17. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
  18. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java

@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -59,13 +61,11 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.timeline.security.authorize.TimelinePolicyProvider;
 
 import com.google.common.base.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class ApplicationHistoryClientService extends AbstractService implements
     ApplicationHistoryProtocol {
-  private static final Logger LOG =
-          LoggerFactory.getLogger(ApplicationHistoryClientService.class);
+  private static final Log LOG = LogFactory
+    .getLog(ApplicationHistoryClientService.class);
   private ApplicationHistoryManager history;
   private Server server;
   private InetSocketAddress bindAddress;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java

@@ -23,6 +23,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
@@ -40,13 +42,11 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.records.Container
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class ApplicationHistoryManagerImpl extends AbstractService implements
     ApplicationHistoryManager {
-  private static final Logger LOG =
-          LoggerFactory.getLogger(ApplicationHistoryManagerImpl.class);
+  private static final Log LOG = LogFactory
+    .getLog(ApplicationHistoryManagerImpl.class);
   private static final String UNAVAILABLE = "N/A";
 
   private ApplicationHistoryStore historyStore;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java

@@ -28,6 +28,8 @@ import java.util.List;
 import java.util.Map;
 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.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -67,14 +69,12 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
     implements
     ApplicationHistoryManager {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(ApplicationHistoryManagerOnTimelineStore.class);
+  private static final Log LOG = LogFactory
+      .getLog(ApplicationHistoryManagerOnTimelineStore.class);
 
   @VisibleForTesting
   static final String UNAVAILABLE = "N/A";

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java

@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
@@ -58,8 +60,6 @@ import org.eclipse.jetty.servlet.FilterHolder;
 import org.eclipse.jetty.webapp.WebAppContext;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * History server that keeps track of all types of history in the cluster.
@@ -68,8 +68,8 @@ import org.slf4j.LoggerFactory;
 public class ApplicationHistoryServer extends CompositeService {
 
   public static final int SHUTDOWN_HOOK_PRIORITY = 30;
-  private static final Logger LOG = LoggerFactory
-      .getLogger(ApplicationHistoryServer.class);
+  private static final Log LOG = LogFactory
+    .getLog(ApplicationHistoryServer.class);
 
   private ApplicationHistoryClientService ahsClientService;
   private ApplicationACLsManager aclsManager;
@@ -178,7 +178,7 @@ public class ApplicationHistoryServer extends CompositeService {
       appHistoryServer.init(conf);
       appHistoryServer.start();
     } catch (Throwable t) {
-      LOG.error("Error starting ApplicationHistoryServer", t);
+      LOG.fatal("Error starting ApplicationHistoryServer", t);
       ExitUtil.terminate(-1, "Error starting ApplicationHistoryServer");
     }
     return appHistoryServer;

+ 11 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java

@@ -30,6 +30,8 @@ import java.util.Map.Entry;
 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.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -72,8 +74,6 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.C
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 import com.google.protobuf.InvalidProtocolBufferException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * File system implementation of {@link ApplicationHistoryStore}. In this
@@ -89,8 +89,8 @@ import org.slf4j.LoggerFactory;
 public class FileSystemApplicationHistoryStore extends AbstractService
     implements ApplicationHistoryStore {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(FileSystemApplicationHistoryStore.class);
+  private static final Log LOG = LogFactory
+    .getLog(FileSystemApplicationHistoryStore.class);
 
   private static final String ROOT_DIR_NAME = "ApplicationHistoryDataRoot";
   private static final int MIN_BLOCK_SIZE = 256 * 1024;
@@ -141,7 +141,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
       }
       outstandingWriters.clear();
     } finally {
-      IOUtils.cleanupWithLogger(LOG, fs);
+      IOUtils.cleanup(LOG, fs);
     }
     super.serviceStop();
   }
@@ -711,12 +711,12 @@ public class FileSystemApplicationHistoryStore extends AbstractService
     }
 
     public void reset() throws IOException {
-      IOUtils.cleanupWithLogger(LOG, scanner);
+      IOUtils.cleanup(LOG, scanner);
       scanner = reader.createScanner();
     }
 
     public void close() {
-      IOUtils.cleanupWithLogger(LOG, scanner, reader, fsdis);
+      IOUtils.cleanup(LOG, scanner, reader, fsdis);
     }
 
   }
@@ -740,13 +740,13 @@ public class FileSystemApplicationHistoryStore extends AbstractService
                 YarnConfiguration.DEFAULT_FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE), null,
                 getConfig());
       } catch (IOException e) {
-        IOUtils.cleanupWithLogger(LOG, fsdos);
+        IOUtils.cleanup(LOG, fsdos);
         throw e;
       }
     }
 
     public synchronized void close() {
-      IOUtils.cleanupWithLogger(LOG, writer, fsdos);
+      IOUtils.cleanup(LOG, writer, fsdos);
     }
 
     public synchronized void writeHistoryData(HistoryDataKey key, byte[] value)
@@ -756,13 +756,13 @@ public class FileSystemApplicationHistoryStore extends AbstractService
         dos = writer.prepareAppendKey(-1);
         key.write(dos);
       } finally {
-        IOUtils.cleanupWithLogger(LOG, dos);
+        IOUtils.cleanup(LOG, dos);
       }
       try {
         dos = writer.prepareAppendValue(value.length);
         dos.write(value);
       } finally {
-        IOUtils.cleanupWithLogger(LOG, dos);
+        IOUtils.cleanup(LOG, dos);
       }
     }
 

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java

@@ -42,6 +42,8 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
+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;
@@ -78,15 +80,12 @@ import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import com.sun.jersey.api.client.ClientHandlerException;
 import com.sun.jersey.api.client.UniformInterfaceException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Singleton
 @Path("/ws/v1/applicationhistory")
 public class AHSWebServices extends WebServices {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(AHSWebServices.class);
+  private static final Log LOG = LogFactory.getLog(AHSWebServices.class);
   private static final String NM_DOWNLOAD_URI_STR =
       "/ws/v1/node/containers";
   private static final Joiner JOINER = Joiner.on("");

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java

@@ -18,20 +18,20 @@
 
 package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class NavBlock extends HtmlBlock {
 
   @Override
   public void render(Block html) {
     boolean addErrorsAndWarningsLink = false;
-    Logger log = LoggerFactory.getLogger(NavBlock.class);
+    Log log = LogFactory.getLog(NavBlock.class);
     if (log instanceof Log4JLogger) {
       Log4jWarningErrorMetricsAppender appender =
           Log4jWarningErrorMetricsAppender.findAppender();

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.timeline;
 
+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.service.AbstractService;
@@ -31,8 +33,6 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEnt
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -71,8 +71,8 @@ abstract class KeyValueBasedTimelineStore
 
   private boolean serviceStopped = false;
 
-  private static final Logger LOG
-      = LoggerFactory.getLogger(KeyValueBasedTimelineStore.class);
+  private static final Log LOG
+      = LogFactory.getLog(KeyValueBasedTimelineStore.class);
 
   public KeyValueBasedTimelineStore() {
     super(KeyValueBasedTimelineStore.class.getName());

+ 18 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java

@@ -22,6 +22,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -46,7 +48,6 @@ import org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.KeyParser;
 import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
 import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.*;
-import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -117,8 +118,8 @@ import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 @InterfaceStability.Unstable
 public class LeveldbTimelineStore extends AbstractService
     implements TimelineStore {
-  private static final org.slf4j.Logger LOG = LoggerFactory
-      .getLogger(LeveldbTimelineStore.class);
+  private static final Log LOG = LogFactory
+      .getLog(LeveldbTimelineStore.class);
 
   @Private
   @VisibleForTesting
@@ -239,7 +240,7 @@ public class LeveldbTimelineStore extends AbstractService
         localFS.setPermission(dbPath, LEVELDB_DIR_UMASK);
       }
     } finally {
-      IOUtils.cleanupWithLogger(LOG, localFS);
+      IOUtils.cleanup(LOG, localFS);
     }
     LOG.info("Using leveldb path " + dbPath);
     try {
@@ -283,7 +284,7 @@ public class LeveldbTimelineStore extends AbstractService
             " closing db now", e);
       }
     }
-    IOUtils.cleanupWithLogger(LOG, db);
+    IOUtils.cleanup(LOG, db);
     super.serviceStop();
   }
 
@@ -319,7 +320,7 @@ public class LeveldbTimelineStore extends AbstractService
           discardOldEntities(timestamp);
           Thread.sleep(ttlInterval);
         } catch (IOException e) {
-          LOG.error(e.toString());
+          LOG.error(e);
         } catch (InterruptedException e) {
           LOG.info("Deletion thread received interrupt, exiting");
           break;
@@ -393,7 +394,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
   }
 
@@ -569,7 +570,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
     return events;
   }
@@ -752,7 +753,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);   	
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
   }
   
@@ -924,7 +925,7 @@ public class LeveldbTimelineStore extends AbstractService
     } finally {
       lock.unlock();
       writeLocks.returnLock(lock);
-      IOUtils.cleanupWithLogger(LOG, writeBatch);
+      IOUtils.cleanup(LOG, writeBatch);
     }
 
     for (EntityIdentifier relatedEntity : relatedEntitiesWithoutStartTimes) {
@@ -1375,7 +1376,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
   }
 
@@ -1505,7 +1506,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanupWithLogger(LOG, writeBatch);
+      IOUtils.cleanup(LOG, writeBatch);
     }
   }
 
@@ -1547,7 +1548,7 @@ public class LeveldbTimelineStore extends AbstractService
           LOG.error("Got IOException while deleting entities for type " +
               entityType + ", continuing to next type", e);
         } finally {
-          IOUtils.cleanupWithLogger(LOG, iterator, pfIterator);
+          IOUtils.cleanup(LOG, iterator, pfIterator);
           deleteLock.writeLock().unlock();
           if (typeCount > 0) {
             LOG.info("Deleted " + typeCount + " entities of type " +
@@ -1628,7 +1629,7 @@ public class LeveldbTimelineStore extends AbstractService
       String incompatibleMessage = 
           "Incompatible version for timeline store: expecting version " 
               + getCurrentVersion() + ", but loading version " + loadedVersion;
-      LOG.error(incompatibleMessage);
+      LOG.fatal(incompatibleMessage);
       throw new IOException(incompatibleMessage);
     }
   }
@@ -1717,7 +1718,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanupWithLogger(LOG, writeBatch);
+      IOUtils.cleanup(LOG, writeBatch);
     }
   }
 
@@ -1754,7 +1755,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
   }
 
@@ -1804,7 +1805,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
   }
 

+ 7 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDB.java

@@ -33,6 +33,8 @@ import java.util.Map.Entry;
 
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.lang.time.FastDateFormat;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -43,8 +45,6 @@ import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.Options;
 import org.iq80.leveldb.WriteBatch;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Contains the logic to lookup a leveldb by timestamp so that multiple smaller
@@ -54,8 +54,7 @@ import org.slf4j.LoggerFactory;
 class RollingLevelDB {
 
   /** Logger for this class. */
-  private static final Logger LOG = LoggerFactory.
-      getLogger(RollingLevelDB.class);
+  private static final Log LOG = LogFactory.getLog(RollingLevelDB.class);
   /** Factory to open and create new leveldb instances. */
   private static JniDBFactory factory = new JniDBFactory();
   /** Thread safe date formatter. */
@@ -152,7 +151,7 @@ class RollingLevelDB {
     }
 
     public void close() {
-      IOUtils.cleanupWithLogger(LOG, writeBatch);
+      IOUtils.cleanup(LOG, writeBatch);
     }
   }
 
@@ -347,7 +346,7 @@ class RollingLevelDB {
         .iterator();
     while (iterator.hasNext()) {
       Entry<Long, DB> entry = iterator.next();
-      IOUtils.cleanupWithLogger(LOG, entry.getValue());
+      IOUtils.cleanup(LOG, entry.getValue());
       String dbName = fdf.format(entry.getKey());
       Path path = new Path(rollingDBPath, getName() + "." + dbName);
       try {
@@ -362,9 +361,9 @@ class RollingLevelDB {
 
   public void stop() throws Exception {
     for (DB db : rollingdbs.values()) {
-      IOUtils.cleanupWithLogger(LOG, db);
+      IOUtils.cleanup(LOG, db);
     }
-    IOUtils.cleanupWithLogger(LOG, lfs);
+    IOUtils.cleanup(LOG, lfs);
   }
 
   private long computeNextCheckMillis(long now) {

+ 11 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java

@@ -38,6 +38,8 @@ import java.util.TreeMap;
 
 import org.apache.commons.collections.map.LRUMap;
 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.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -74,8 +76,6 @@ import org.iq80.leveldb.ReadOptions;
 import org.iq80.leveldb.WriteBatch;
 import org.nustaq.serialization.FSTConfiguration;
 import org.nustaq.serialization.FSTClazzNameRegistry;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -168,8 +168,8 @@ import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 @InterfaceStability.Unstable
 public class RollingLevelDBTimelineStore extends AbstractService implements
     TimelineStore {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(RollingLevelDBTimelineStore.class);
+  private static final Log LOG = LogFactory
+      .getLog(RollingLevelDBTimelineStore.class);
   private static FSTConfiguration fstConf =
       FSTConfiguration.createDefaultConfiguration();
   // Fall back to 2.24 parsing if 2.50 parsing fails
@@ -368,9 +368,9 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
             + " closing db now", e);
       }
     }
-    IOUtils.cleanupWithLogger(LOG, domaindb);
-    IOUtils.cleanupWithLogger(LOG, starttimedb);
-    IOUtils.cleanupWithLogger(LOG, ownerdb);
+    IOUtils.cleanup(LOG, domaindb);
+    IOUtils.cleanup(LOG, starttimedb);
+    IOUtils.cleanup(LOG, ownerdb);
     entitydb.stop();
     indexdb.stop();
     super.serviceStop();
@@ -399,7 +399,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
           discardOldEntities(timestamp);
           Thread.sleep(ttlInterval);
         } catch (IOException e) {
-          LOG.error(e.toString());
+          LOG.error(e);
         } catch (InterruptedException e) {
           LOG.info("Deletion thread received interrupt, exiting");
           break;
@@ -1525,7 +1525,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
                   + ". Total start times deleted so far this cycle: "
                   + startTimesCount);
             }
-            IOUtils.cleanupWithLogger(LOG, writeBatch);
+            IOUtils.cleanup(LOG, writeBatch);
             writeBatch = starttimedb.createWriteBatch();
             batchSize = 0;
           }
@@ -1545,7 +1545,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
       LOG.info("Deleted " + startTimesCount + "/" + totalCount
           + " start time entities earlier than " + minStartTime);
     } finally {
-      IOUtils.cleanupWithLogger(LOG, writeBatch);
+      IOUtils.cleanup(LOG, writeBatch);
     }
     return startTimesCount;
   }
@@ -1622,7 +1622,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
       String incompatibleMessage = "Incompatible version for timeline store: "
           + "expecting version " + getCurrentVersion()
           + ", but loading version " + loadedVersion;
-      LOG.error(incompatibleMessage);
+      LOG.fatal(incompatibleMessage);
       throw new IOException(incompatibleMessage);
     }
   }

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java

@@ -26,6 +26,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.SortedSet;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
@@ -43,8 +45,6 @@ import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * The class wrap over the timeline store and the ACLs manager. It does some non
@@ -54,8 +54,7 @@ import org.slf4j.LoggerFactory;
  */
 public class TimelineDataManager extends AbstractService {
 
-  private static final Logger LOG =
-          LoggerFactory.getLogger(TimelineDataManager.class);
+  private static final Log LOG = LogFactory.getLog(TimelineDataManager.class);
   @VisibleForTesting
   public static final String DEFAULT_DOMAIN_ID = "DEFAULT";
 

+ 15 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/recovery/LeveldbTimelineStateStore.java

@@ -28,6 +28,8 @@ import java.io.File;
 import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -48,8 +50,6 @@ import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DBException;
 import org.iq80.leveldb.Options;
 import org.iq80.leveldb.WriteBatch;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 
@@ -60,8 +60,8 @@ import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 public class LeveldbTimelineStateStore extends
     TimelineStateStore {
 
-  public static final Logger LOG =
-      LoggerFactory.getLogger(LeveldbTimelineStateStore.class);
+  public static final Log LOG =
+      LogFactory.getLog(LeveldbTimelineStateStore.class);
 
   private static final String DB_NAME = "timeline-state-store.ldb";
   private static final FsPermission LEVELDB_DIR_UMASK = FsPermission
@@ -103,7 +103,7 @@ public class LeveldbTimelineStateStore extends
         localFS.setPermission(dbPath, LEVELDB_DIR_UMASK);
       }
     } finally {
-      IOUtils.cleanupWithLogger(LOG, localFS);
+      IOUtils.cleanup(LOG, localFS);
     }
     JniDBFactory factory = new JniDBFactory();
     try {
@@ -131,7 +131,7 @@ public class LeveldbTimelineStateStore extends
 
   @Override
   protected void closeStorage() throws IOException {
-    IOUtils.cleanupWithLogger(LOG, db);
+    IOUtils.cleanup(LOG, db);
   }
 
   @Override
@@ -168,8 +168,8 @@ public class LeveldbTimelineStateStore extends
     } catch (DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanupWithLogger(LOG, ds);
-      IOUtils.cleanupWithLogger(LOG, batch);
+      IOUtils.cleanup(LOG, ds);
+      IOUtils.cleanup(LOG, batch);
     }
   }
 
@@ -239,7 +239,7 @@ public class LeveldbTimelineStateStore extends
       key.write(dataStream);
       dataStream.close();
     } finally {
-      IOUtils.cleanupWithLogger(LOG, dataStream);
+      IOUtils.cleanup(LOG, dataStream);
     }
     return memStream.toByteArray();
   }
@@ -253,7 +253,7 @@ public class LeveldbTimelineStateStore extends
     try {
       key.readFields(in);
     } finally {
-      IOUtils.cleanupWithLogger(LOG, in);
+      IOUtils.cleanup(LOG, in);
     }
     state.tokenMasterKeyState.add(key);
   }
@@ -267,7 +267,7 @@ public class LeveldbTimelineStateStore extends
     try {
       data.readFields(in);
     } finally {
-      IOUtils.cleanupWithLogger(LOG, in);
+      IOUtils.cleanup(LOG, in);
     }
     state.tokenState.put(data.getTokenIdentifier(), data.getRenewDate());
   }
@@ -290,7 +290,7 @@ public class LeveldbTimelineStateStore extends
         ++numKeys;
       }
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
     return numKeys;
   }
@@ -314,7 +314,7 @@ public class LeveldbTimelineStateStore extends
     } catch (DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanupWithLogger(LOG, iterator);
+      IOUtils.cleanup(LOG, iterator);
     }
     return numTokens;
   }
@@ -332,7 +332,7 @@ public class LeveldbTimelineStateStore extends
       try {
         state.latestSequenceNumber = in.readInt();
       } finally {
-        IOUtils.cleanupWithLogger(LOG, in);
+        IOUtils.cleanup(LOG, in);
       }
     }
   }
@@ -412,7 +412,7 @@ public class LeveldbTimelineStateStore extends
       String incompatibleMessage =
           "Incompatible version for timeline state store: expecting version "
               + getCurrentVersion() + ", but loading version " + loadedVersion;
-      LOG.error(incompatibleMessage);
+      LOG.fatal(incompatibleMessage);
       throw new IOException(incompatibleMessage);
     }
   }

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java

@@ -24,6 +24,8 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.commons.collections.map.LRUMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -39,8 +41,6 @@ import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.util.StringHelper;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * <code>TimelineACLsManager</code> check the entity level timeline data access.
@@ -48,8 +48,7 @@ import org.slf4j.LoggerFactory;
 @Private
 public class TimelineACLsManager {
 
-  private static final Logger LOG = LoggerFactory.
-      getLogger(TimelineACLsManager.class);
+  private static final Log LOG = LogFactory.getLog(TimelineACLsManager.class);
   private static final int DOMAIN_ACCESS_ENTRY_CACHE_SIZE = 100;
 
   private AdminACLsManager adminAclsManager;

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.timeline.security;
 import java.io.IOException;
 import java.util.Map.Entry;
 
+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;
@@ -33,8 +35,6 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore.TimelineServiceState;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * The service wrapper of {@link TimelineDelegationTokenSecretManager}
@@ -118,8 +118,8 @@ public class TimelineDelegationTokenSecretManagerService extends
   public static class TimelineDelegationTokenSecretManager extends
       AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
 
-    public static final Logger LOG =
-        LoggerFactory.getLogger(TimelineDelegationTokenSecretManager.class);
+    public static final Log LOG =
+        LogFactory.getLog(TimelineDelegationTokenSecretManager.class);
 
     private TimelineStateStore stateStore;
 

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java

@@ -43,6 +43,8 @@ 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.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
@@ -66,16 +68,13 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @Singleton
 @Path("/ws/v1/timeline")
 //TODO: support XML serialization/deserialization
 public class TimelineWebServices {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TimelineWebServices.class);
+  private static final Log LOG = LogFactory.getLog(TimelineWebServices.class);
 
   private TimelineDataManager timelineDataManager;
 

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java

@@ -32,6 +32,8 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,14 +51,12 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class TestFileSystemApplicationHistoryStore extends
     ApplicationHistoryStoreTestUtils {
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestFileSystemApplicationHistoryStore.class.getName());
+  private static Log LOG = LogFactory
+    .getLog(TestFileSystemApplicationHistoryStore.class.getName());
 
   private FileSystem fs;
   private Path fsWorkingPath;

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java

@@ -160,7 +160,7 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     } catch(DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanupWithLogger(null, iterator, pfIterator);
+      IOUtils.cleanup(null, iterator, pfIterator);
     }
   }