ソースを参照

HADOOP-18654. Remove unused custom appender TaskLogAppender (#5457)

Viraj Jasani 2 年 前
コミット
405bfa2800

+ 0 - 8
hadoop-common-project/hadoop-common/src/main/conf/log4j.properties

@@ -75,14 +75,6 @@ log4j.appender.console.target=System.err
 log4j.appender.console.layout=org.apache.log4j.PatternLayout
 log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
 
-#
-# TaskLog Appender
-#
-log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender
-
-log4j.appender.TLA.layout=org.apache.log4j.PatternLayout
-log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
-
 #
 # HDFS block state change log from block manager
 #

+ 0 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java

@@ -256,17 +256,6 @@ public class TaskLog {
   throws IOException {
     System.out.flush();
     System.err.flush();
-    Enumeration<Logger> allLoggers = LogManager.getCurrentLoggers();
-    while (allLoggers.hasMoreElements()) {
-      Logger l = allLoggers.nextElement();
-      Enumeration<Appender> allAppenders = l.getAllAppenders();
-      while (allAppenders.hasMoreElements()) {
-        Appender a = allAppenders.nextElement();
-        if (a instanceof TaskLogAppender) {
-          ((TaskLogAppender)a).flush();
-        }
-      }
-    }
     if (currentTaskid != taskid) {
       currentTaskid = taskid;
       resetPrevLengths(logLocation);

+ 0 - 153
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLogAppender.java

@@ -1,153 +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.mapred;
-
-import java.io.Flushable;
-import java.util.LinkedList;
-import java.util.Queue;
-
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.log4j.FileAppender;
-import org.apache.log4j.spi.LoggingEvent;
-
-/**
- * A simple log4j-appender for the task child's 
- * map-reduce system logs.
- * 
- */
-@InterfaceStability.Unstable
-public class TaskLogAppender extends FileAppender implements Flushable {
-  private String taskId; //taskId should be managed as String rather than TaskID object
-  //so that log4j can configure it from the configuration(log4j.properties). 
-  private Integer maxEvents;
-  private Queue<LoggingEvent> tail = null;
-  private Boolean isCleanup;
-
-  // System properties passed in from JVM runner
-  static final String ISCLEANUP_PROPERTY = "hadoop.tasklog.iscleanup";
-  static final String LOGSIZE_PROPERTY = "hadoop.tasklog.totalLogFileSize";
-  static final String TASKID_PROPERTY = "hadoop.tasklog.taskid";
-
-  @Override
-  public void activateOptions() {
-    synchronized (this) {
-      setOptionsFromSystemProperties();
-
-      if (maxEvents > 0) {
-        tail = new LinkedList<LoggingEvent>();
-      }
-      setFile(TaskLog.getTaskLogFile(TaskAttemptID.forName(taskId),
-          isCleanup, TaskLog.LogName.SYSLOG).toString());
-      setAppend(true);
-      super.activateOptions();
-    }
-  }
-
-  /**
-   * The Task Runner passes in the options as system properties. Set
-   * the options if the setters haven't already been called.
-   */
-  private synchronized void setOptionsFromSystemProperties() {
-    if (isCleanup == null) {
-      String propValue = System.getProperty(ISCLEANUP_PROPERTY, "false");
-      isCleanup = Boolean.valueOf(propValue);
-    }
-
-    if (taskId == null) {
-      taskId = System.getProperty(TASKID_PROPERTY);
-    }
-
-    if (maxEvents == null) {
-      String propValue = System.getProperty(LOGSIZE_PROPERTY, "0");
-      setTotalLogFileSize(Long.parseLong(propValue));
-    }
-  }
-  
-  @Override
-  public void append(LoggingEvent event) {
-    synchronized (this) {
-      if (tail == null) {
-        super.append(event);
-      } else {
-        if (tail.size() >= maxEvents) {
-          tail.remove();
-        }
-        tail.add(event);
-      }
-    }
-  }
-  
-  @Override
-  public void flush() {
-    if (qw != null) {
-      qw.flush();
-    }
-  }
-
-  @Override
-  public synchronized void close() {
-    if (tail != null) {
-      for(LoggingEvent event: tail) {
-        super.append(event);
-      }
-    }
-    super.close();
-  }
-
-  /**
-   * Getter/Setter methods for log4j.
-   */
-  
-  public synchronized String getTaskId() {
-    return taskId;
-  }
-
-  public synchronized void setTaskId(String taskId) {
-    this.taskId = taskId;
-  }
-
-  private static final int EVENT_SIZE = 100;
-  
-  public synchronized long getTotalLogFileSize() {
-    return maxEvents * EVENT_SIZE;
-  }
-
-  public synchronized void setTotalLogFileSize(long logSize) {
-    maxEvents = (int) logSize / EVENT_SIZE;
-  }
-
-  /**
-   * Set whether the task is a cleanup attempt or not.
-   * 
-   * @param isCleanup
-   *          true if the task is cleanup attempt, false otherwise.
-   */
-  public synchronized void setIsCleanup(boolean isCleanup) {
-    this.isCleanup = isCleanup;
-  }
-
-  /**
-   * Get whether task is cleanup attempt or not.
-   * 
-   * @return true if the task is cleanup attempt, false otherwise.
-   */
-  public synchronized boolean getIsCleanup() {
-    return isCleanup;
-  }
-}

+ 0 - 74
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskLogAppender.java

@@ -1,74 +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.mapred;
-
-import java.io.StringWriter;
-import java.io.Writer;
-
-import org.apache.log4j.Category;
-import org.apache.log4j.Layout;
-import org.apache.log4j.Logger;
-import org.apache.log4j.PatternLayout;
-import org.apache.log4j.Priority;
-import org.apache.log4j.spi.LoggingEvent;
-import org.junit.Test;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class TestTaskLogAppender {
-/**
- * test TaskLogAppender 
- */
-  @SuppressWarnings("deprecation")
-  @Test (timeout=5000)
-  public void testTaskLogAppender(){
-    TaskLogAppender appender= new TaskLogAppender();
-    
-    System.setProperty(TaskLogAppender.TASKID_PROPERTY,"attempt_01_02_m03_04_001");
-    System.setProperty(TaskLogAppender.LOGSIZE_PROPERTY, "1003");
-    appender.activateOptions();
-    assertThat(appender.getTaskId()).isEqualTo("attempt_01_02_m03_04_001");
-    assertThat(appender.getTotalLogFileSize()).isEqualTo(1000);
-    assertFalse(appender.getIsCleanup());
-    
-    // test writer   
-    Writer writer= new StringWriter();
-    appender.setWriter(writer);
-    Layout layout =  new PatternLayout("%-5p [%t]: %m%n");
-    appender.setLayout(layout);
-    Category logger= Logger.getLogger(getClass().getName());
-    LoggingEvent event = new LoggingEvent("fqnOfCategoryClass", logger, Priority.INFO, "message", new Throwable());
-    appender.append(event);
-    appender.flush() ;
-    appender.close();
-    assertTrue(writer.toString().length()>0);
-    
-    // test cleanup should not changed 
-    appender= new TaskLogAppender();
-    appender.setIsCleanup(true);
-    appender.activateOptions();
-    assertTrue(appender.getIsCleanup());
-
-  
-  }
-  
-}