You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by ni...@apache.org on 2015/10/07 01:44:20 UTC

samza git commit: SAMZA-787: task.log4j.system should not be implicitly inferred

Repository: samza
Updated Branches:
  refs/heads/master eb80cfee8 -> 64f3f6b32


SAMZA-787: task.log4j.system should not be implicitly inferred


Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/64f3f6b3
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/64f3f6b3
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/64f3f6b3

Branch: refs/heads/master
Commit: 64f3f6b32d024172c9dcc0309af314dd0fb620d4
Parents: eb80cfe
Author: Navina Ramesh <na...@gmail.com>
Authored: Tue Oct 6 16:44:10 2015 -0700
Committer: Yi Pan (Data Infrastructure) <yi...@linkedin.com>
Committed: Tue Oct 6 16:44:10 2015 -0700

----------------------------------------------------------------------
 .../versioned/jobs/configuration-table.html            |  5 ++---
 docs/learn/documentation/versioned/jobs/logging.md     |  9 +++++++--
 .../org/apache/samza/config/Log4jSystemConfig.java     | 13 +++----------
 .../org/apache/samza/logging/log4j/StreamAppender.java |  2 +-
 .../org/apache/samza/config/TestLog4jSystemConfig.java |  7 +------
 .../apache/samza/logging/log4j/TestStreamAppender.java |  2 ++
 6 files changed, 16 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/64f3f6b3/docs/learn/documentation/versioned/jobs/configuration-table.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html b/docs/learn/documentation/versioned/jobs/configuration-table.html
index b42c34c..ddfe836 100644
--- a/docs/learn/documentation/versioned/jobs/configuration-table.html
+++ b/docs/learn/documentation/versioned/jobs/configuration-table.html
@@ -381,9 +381,8 @@
                     <td class="property" id="task-log4j-system">task.log4j.system</td>
                     <td class="default"></td>
                     <td class="description">
-                        Specify the system name for the StreamAppender. If Samza can not find this property and there is
-                        only one system in the config, it will guess to use that system for the log4j appender. If this property
-                        is not specified and there are more than one system in the config, Samza throws exception. (See
+                        Specify the system name for the StreamAppender. If this property is not specified in the config,
+                        Samza throws exception. (See
                         <a href="logging.html#stream-log4j-appender">Stream Log4j Appender</a>)
                         <dl>
                             <dt>Example: <code>task.log4j.system=kafka</code></dt>

http://git-wip-us.apache.org/repos/asf/samza/blob/64f3f6b3/docs/learn/documentation/versioned/jobs/logging.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/logging.md b/docs/learn/documentation/versioned/jobs/logging.md
index a3bb054..caca43f 100644
--- a/docs/learn/documentation/versioned/jobs/logging.md
+++ b/docs/learn/documentation/versioned/jobs/logging.md
@@ -97,7 +97,7 @@ And then updating your log4j.xml to include the appender:
 
 #### Stream Log4j Appender
 
-Samza provides a StreamAppender to publish the logs into a specific system. You can specify the system name using "task.log4j.system" and change name of log stream with param 'StreamName'. If there is only one system in the config, Samza will use that system for the log publishing. Also we have the [MDC](http://logback.qos.ch/manual/mdc.html) keys "containerName", "jobName" and "jobId", which help identify the source of the log. In order to use this appender, simply add:
+Samza provides a StreamAppender to publish the logs into a specific system. You can specify the system name using "task.log4j.system" and change name of log stream with param 'StreamName'. Also, we have the [MDC](http://logback.qos.ch/manual/mdc.html) keys "containerName", "jobName" and "jobId", which help identify the source of the log. In order to use this appender, simply add:
 
 {% highlight xml %}
 <appender name="StreamAppender" class="org.apache.samza.logging.log4j.StreamAppender">
@@ -109,12 +109,17 @@ Samza provides a StreamAppender to publish the logs into a specific system. You
 </appender>
 {% endhighlight %}
 
-and add:
+and
 
 {% highlight xml %}
 <appender-ref ref="StreamAppender"/>
 {% endhighlight %}
 
+to log4j.xml and define the system name by specifying the config:
+{% highlight xml %}
+task.log4j.system="<system-name>"
+{% endhighlight %}
+
 Configuring the StreamAppender will automatically encode messages using logstash's [Log4J JSON format](https://github.com/logstash/log4j-jsonevent-layout). Samza also supports pluggable serialization for those that prefer non-JSON logging events. This can be configured the same way other stream serializers are defined:
 
 {% highlight jproperties %}

http://git-wip-us.apache.org/repos/asf/samza/blob/64f3f6b3/samza-log4j/src/main/java/org/apache/samza/config/Log4jSystemConfig.java
----------------------------------------------------------------------
diff --git a/samza-log4j/src/main/java/org/apache/samza/config/Log4jSystemConfig.java b/samza-log4j/src/main/java/org/apache/samza/config/Log4jSystemConfig.java
index 209296d..d98b8c6 100644
--- a/samza-log4j/src/main/java/org/apache/samza/config/Log4jSystemConfig.java
+++ b/samza-log4j/src/main/java/org/apache/samza/config/Log4jSystemConfig.java
@@ -19,8 +19,6 @@
 
 package org.apache.samza.config;
 
-import java.util.List;
-
 /**
  * This class contains the methods for getting properties that are needed by the
  * StreamAppender.
@@ -47,20 +45,15 @@ public class Log4jSystemConfig extends JavaSystemConfig {
   }
 
   /**
-   * Get the log4j system name from the config. If it's not defined, try to
-   * guess the system name if there is only one system is defined.
+   * Get the log4j system name from the config.
+   * If it's not defined, throw a ConfigException
    *
    * @return log4j system name
    */
   public String getSystemName() {
     String log4jSystem = get(TASK_LOG4J_SYSTEM, null);
     if (log4jSystem == null) {
-      List<String> systemNames = getSystemNames();
-      if (systemNames.size() == 1) {
-        log4jSystem = systemNames.get(0);
-      } else {
-        throw new ConfigException("Missing " + TASK_LOG4J_SYSTEM + " configuration, and more than 1 systems were found.");
-      }
+      throw new ConfigException("Missing " + TASK_LOG4J_SYSTEM + " configuration. Can't figure out the system name to use.");
     }
     return log4jSystem;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/64f3f6b3/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java
----------------------------------------------------------------------
diff --git a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java
index 8948453..776a36b 100644
--- a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java
+++ b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/StreamAppender.java
@@ -104,7 +104,7 @@ public class StreamAppender extends AppenderSkeleton {
     if (systemFactoryName != null) {
       systemFactory = Util.<SystemFactory>getObj(systemFactoryName);
     } else {
-      throw new SamzaException("Please define log4j system name and factory class");
+      throw new SamzaException("Could not figure out the \"" + systemName + "\" system factory for log4j StreamAppender to use");
     }
 
     setSerde(log4jSystemConfig, systemName, streamName);

http://git-wip-us.apache.org/repos/asf/samza/blob/64f3f6b3/samza-log4j/src/test/java/org/apache/samza/config/TestLog4jSystemConfig.java
----------------------------------------------------------------------
diff --git a/samza-log4j/src/test/java/org/apache/samza/config/TestLog4jSystemConfig.java b/samza-log4j/src/test/java/org/apache/samza/config/TestLog4jSystemConfig.java
index f7d3cbe..a1eeb38 100644
--- a/samza-log4j/src/test/java/org/apache/samza/config/TestLog4jSystemConfig.java
+++ b/samza-log4j/src/test/java/org/apache/samza/config/TestLog4jSystemConfig.java
@@ -53,13 +53,8 @@ public class TestLog4jSystemConfig {
     Log4jSystemConfig log4jSystemConfig = new Log4jSystemConfig(new MapConfig(map));
     assertEquals("log4j-system", log4jSystemConfig.getSystemName());
 
-    // use the default system name
-    map.remove("task.log4j.system");
-    log4jSystemConfig = new Log4jSystemConfig(new MapConfig(map));
-    assertEquals("system1", log4jSystemConfig.getSystemName());
-
     // throw ConfigException
-    map.put("systems.system2.samza.factory", "2");
+    map.remove("task.log4j.system");
     log4jSystemConfig = new Log4jSystemConfig(new MapConfig(map));
     exception.expect(ConfigException.class);
     log4jSystemConfig.getSystemName();

http://git-wip-us.apache.org/repos/asf/samza/blob/64f3f6b3/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java
----------------------------------------------------------------------
diff --git a/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java b/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java
index 3e81240..1c6f9a4 100644
--- a/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java
+++ b/samza-log4j/src/test/java/org/apache/samza/logging/log4j/TestStreamAppender.java
@@ -59,6 +59,7 @@ public class TestStreamAppender {
     map.put("serializers.registry.log4j-string.class", LoggingEventStringSerdeFactory.class.getCanonicalName());
     map.put("systems.mock.samza.factory", MockSystemFactory.class.getCanonicalName());
     map.put("systems.mock.streams." + streamName + ".samza.msg.serde", "log4j-string");
+    map.put("task.log4j.system", "mock");
     MockSystemProducerAppender systemProducerAppender = new MockSystemProducerAppender(new MapConfig(map));
     PatternLayout layout = new PatternLayout();
     layout.setConversionPattern("%m");
@@ -100,6 +101,7 @@ public class TestStreamAppender {
       Map<String, String> map = new HashMap<String, String>();
       map.put("job.name", "log4jTest");
       map.put("systems.mock.samza.factory", MockSystemFactory.class.getCanonicalName());
+      map.put("task.log4j.system", "mock");
       config = new MapConfig(map);
     }