kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ewe...@apache.org
Subject kafka git commit: KAFKA-2927; reduce system test storage footprint
Date Fri, 11 Dec 2015 01:35:17 GMT
Repository: kafka
Updated Branches:
  refs/heads/trunk a5f1537f8 -> 18d7abd89


KAFKA-2927; reduce system test storage footprint

Split kafka logging into two levels - DEBUG and INFO, and do not collect DEBUG by default.

Author: Geoff Anderson <geoff@confluent.io>

Reviewers: Ben Stopford <ben@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #657 from granders/KAFKA-2927-reduce-log-footprint


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/18d7abd8
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/18d7abd8
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/18d7abd8

Branch: refs/heads/trunk
Commit: 18d7abd899c5bdbbcde3be448f24458ed4f9a851
Parents: a5f1537
Author: Geoff Anderson <geoff@confluent.io>
Authored: Thu Dec 10 17:35:03 2015 -0800
Committer: Ewen Cheslack-Postava <me@ewencp.org>
Committed: Thu Dec 10 17:35:03 2015 -0800

----------------------------------------------------------------------
 tests/kafkatest/services/kafka/kafka.py         |  12 +-
 .../services/kafka/templates/log4j.properties   | 152 ++++++++++++-------
 tests/kafkatest/services/verifiable_producer.py |   1 -
 .../kafkatest/tests/produce_consume_validate.py |  16 +-
 4 files changed, 118 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/18d7abd8/tests/kafkatest/services/kafka/kafka.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py
index 5fee147..4760065 100644
--- a/tests/kafkatest/services/kafka/kafka.py
+++ b/tests/kafkatest/services/kafka/kafka.py
@@ -42,15 +42,19 @@ class KafkaService(JmxMixin, Service):
     LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "kafka-log4j.properties")
     # Logs such as controller.log, server.log, etc all go here
     OPERATIONAL_LOG_DIR = os.path.join(PERSISTENT_ROOT, "kafka-operational-logs")
+    OPERATIONAL_LOG_INFO_DIR = os.path.join(OPERATIONAL_LOG_DIR, "info")
+    OPERATIONAL_LOG_DEBUG_DIR = os.path.join(OPERATIONAL_LOG_DIR, "debug")
     # Kafka log segments etc go here
     DATA_LOG_DIR = os.path.join(PERSISTENT_ROOT, "kafka-data-logs")
     CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "kafka.properties")
 
-
     logs = {
-        "kafka_operational_logs": {
-            "path": OPERATIONAL_LOG_DIR,
+        "kafka_operational_logs_info": {
+            "path": OPERATIONAL_LOG_INFO_DIR,
             "collect_default": True},
+        "kafka_operational_logs_debug": {
+            "path": OPERATIONAL_LOG_DEBUG_DIR,
+            "collect_default": False},
         "kafka_data": {
             "path": DATA_LOG_DIR,
             "collect_default": False}
@@ -66,7 +70,6 @@ class KafkaService(JmxMixin, Service):
         """
         Service.__init__(self, context, num_nodes)
         JmxMixin.__init__(self, num_nodes, jmx_object_names, jmx_attributes)
-        self.log_level = "DEBUG"
 
         self.zk = zk
         self.quota_config = quota_config
@@ -165,7 +168,6 @@ class KafkaService(JmxMixin, Service):
     def start_cmd(self, node):
         cmd = "export JMX_PORT=%d; " % self.jmx_port
         cmd += "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG
-        cmd += "export LOG_DIR=%s; " % KafkaService.OPERATIONAL_LOG_DIR
         cmd += "export KAFKA_OPTS=%s; " % self.security_config.kafka_opts
         cmd += "/opt/" + kafka_dir(node) + "/bin/kafka-server-start.sh %s 1>> %s 2>>
%s &" % (KafkaService.CONFIG_FILE, KafkaService.STDOUT_CAPTURE, KafkaService.STDERR_CAPTURE)
         return cmd

http://git-wip-us.apache.org/repos/asf/kafka/blob/18d7abd8/tests/kafkatest/services/kafka/templates/log4j.properties
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/templates/log4j.properties b/tests/kafkatest/services/kafka/templates/log4j.properties
index 904e51e..8098f3b 100644
--- a/tests/kafkatest/services/kafka/templates/log4j.properties
+++ b/tests/kafkatest/services/kafka/templates/log4j.properties
@@ -13,75 +13,125 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-log4j.rootLogger={{ log_level }}, stdout
+log4j.rootLogger=DEBUG, stdout
 
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n
 
-log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.kafkaAppender.DatePattern='.'yyyy-MM-dd-HH
-log4j.appender.kafkaAppender.File={{ log_dir }}/server.log
-log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout
-log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
-
-log4j.appender.stateChangeAppender=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.stateChangeAppender.DatePattern='.'yyyy-MM-dd-HH
-log4j.appender.stateChangeAppender.File={{ log_dir }}/state-change.log
-log4j.appender.stateChangeAppender.layout=org.apache.log4j.PatternLayout
-log4j.appender.stateChangeAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
-
-log4j.appender.requestAppender=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.requestAppender.DatePattern='.'yyyy-MM-dd-HH
-log4j.appender.requestAppender.File={{ log_dir }}/kafka-request.log
-log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout
-log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
-
-log4j.appender.cleanerAppender=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.cleanerAppender.DatePattern='.'yyyy-MM-dd-HH
-log4j.appender.cleanerAppender.File={{ log_dir }}/log-cleaner.log
-log4j.appender.cleanerAppender.layout=org.apache.log4j.PatternLayout
-log4j.appender.cleanerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
-
-log4j.appender.controllerAppender=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.controllerAppender.DatePattern='.'yyyy-MM-dd-HH
-log4j.appender.controllerAppender.File={{ log_dir }}/controller.log
-log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout
-log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
-
-log4j.appender.authorizerAppender=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.authorizerAppender.DatePattern='.'yyyy-MM-dd-HH
-log4j.appender.authorizerAppender.File={{ log_dir }}/kafka-authorizer.log
-log4j.appender.authorizerAppender.layout=org.apache.log4j.PatternLayout
-log4j.appender.authorizerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+# INFO level appenders
+log4j.appender.kafkaInfoAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.kafkaInfoAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.kafkaInfoAppender.File={{ log_dir }}/info/server.log
+log4j.appender.kafkaInfoAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.kafkaInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.kafkaInfoAppender.Threshold=INFO
+
+log4j.appender.stateChangeInfoAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.stateChangeInfoAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.stateChangeInfoAppender.File={{ log_dir }}/info/state-change.log
+log4j.appender.stateChangeInfoAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.stateChangeInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.stateChangeInfoAppender.Threshold=INFO
+
+log4j.appender.requestInfoAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.requestInfoAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.requestInfoAppender.File={{ log_dir }}/info/kafka-request.log
+log4j.appender.requestInfoAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.requestInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.requestInfoAppender.Threshold=INFO
+
+log4j.appender.cleanerInfoAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.cleanerInfoAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.cleanerInfoAppender.File={{ log_dir }}/info/log-cleaner.log
+log4j.appender.cleanerInfoAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.cleanerInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.cleanerInfoAppender.Threshold=INFO
+
+log4j.appender.controllerInfoAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.controllerInfoAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.controllerInfoAppender.File={{ log_dir }}/info/controller.log
+log4j.appender.controllerInfoAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.controllerInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.controllerInfoAppender.Threshold=INFO
+
+log4j.appender.authorizerInfoAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.authorizerInfoAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.authorizerInfoAppender.File={{ log_dir }}/info/kafka-authorizer.log
+log4j.appender.authorizerInfoAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.authorizerInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.authorizerInfoAppender.Threshold=INFO
+
+# DEBUG level appenders
+log4j.appender.kafkaDebugAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.kafkaDebugAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.kafkaDebugAppender.File={{ log_dir }}/debug/server.log
+log4j.appender.kafkaDebugAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.kafkaDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.kafkaDebugAppender.Threshold=DEBUG
+
+log4j.appender.stateChangeDebugAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.stateChangeDebugAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.stateChangeDebugAppender.File={{ log_dir }}/debug/state-change.log
+log4j.appender.stateChangeDebugAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.stateChangeDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.stateChangeDebugAppender.Threshold=DEBUG
+
+log4j.appender.requestDebugAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.requestDebugAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.requestDebugAppender.File={{ log_dir }}/debug/kafka-request.log
+log4j.appender.requestDebugAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.requestDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.requestDebugAppender.Threshold=DEBUG
+
+log4j.appender.cleanerDebugAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.cleanerDebugAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.cleanerDebugAppender.File={{ log_dir }}/debug/log-cleaner.log
+log4j.appender.cleanerDebugAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.cleanerDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.cleanerDebugAppender.Threshold=DEBUG
+
+log4j.appender.controllerDebugAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.controllerDebugAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.controllerDebugAppender.File={{ log_dir }}/debug/controller.log
+log4j.appender.controllerDebugAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.controllerDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.controllerDebugAppender.Threshold=DEBUG
+
+log4j.appender.authorizerDebugAppender=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.authorizerDebugAppender.DatePattern='.'yyyy-MM-dd-HH
+log4j.appender.authorizerDebugAppender.File={{ log_dir }}/debug/kafka-authorizer.log
+log4j.appender.authorizerDebugAppender.layout=org.apache.log4j.PatternLayout
+log4j.appender.authorizerDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
+log4j.appender.authorizerDebugAppender.Threshold=DEBUG
 
 # Turn on all our debugging info
-log4j.logger.kafka.producer.async.DefaultEventHandler={{ log_level }}, kafkaAppender
-log4j.logger.kafka.client.ClientUtils={{ log_level }}, kafkaAppender
-log4j.logger.kafka.perf={{ log_level }}, kafkaAppender
-log4j.logger.kafka.perf.ProducerPerformance$ProducerThread={{ log_level }}, kafkaAppender
-log4j.logger.org.I0Itec.zkclient.ZkClient={{ log_level }}, kafkaAppender
-log4j.logger.kafka={{ log_level }}, kafkaAppender
-
-log4j.logger.kafka.network.RequestChannel$={{ log_level }}, requestAppender
+log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaInfoAppender, kafkaDebugAppender
+log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaInfoAppender, kafkaDebugAppender
+log4j.logger.kafka.perf=DEBUG, kafkaInfoAppender, kafkaDebugAppender
+log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, kafkaInfoAppender, kafkaDebugAppender
+log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG, kafkaInfoAppender, kafkaDebugAppender
+log4j.logger.kafka=DEBUG, kafkaInfoAppender, kafkaDebugAppender
+
+log4j.logger.kafka.network.RequestChannel$=DEBUG, requestInfoAppender, requestDebugAppender
 log4j.additivity.kafka.network.RequestChannel$=false
 
-log4j.logger.kafka.network.Processor={{ log_level }}, requestAppender
-log4j.logger.kafka.server.KafkaApis={{ log_level }}, requestAppender
+log4j.logger.kafka.network.Processor=DEBUG, requestInfoAppender, requestDebugAppender
+log4j.logger.kafka.server.KafkaApis=DEBUG, requestInfoAppender, requestDebugAppender
 log4j.additivity.kafka.server.KafkaApis=false
-log4j.logger.kafka.request.logger={{ log_level }}, requestAppender
+log4j.logger.kafka.request.logger=DEBUG, requestInfoAppender, requestDebugAppender
 log4j.additivity.kafka.request.logger=false
 
-log4j.logger.kafka.controller={{ log_level }}, controllerAppender
+log4j.logger.kafka.controller=DEBUG, controllerInfoAppender, controllerDebugAppender
 log4j.additivity.kafka.controller=false
 
-log4j.logger.kafka.log.LogCleaner={{ log_level }}, cleanerAppender
+log4j.logger.kafka.log.LogCleaner=DEBUG, cleanerInfoAppender, cleanerDebugAppender
 log4j.additivity.kafka.log.LogCleaner=false
 
-log4j.logger.state.change.logger={{ log_level }}, stateChangeAppender
+log4j.logger.state.change.logger=DEBUG, stateChangeInfoAppender, stateChangeDebugAppender
 log4j.additivity.state.change.logger=false
 
 #Change this to debug to get the actual audit log for authorizer.
-log4j.logger.kafka.authorizer.logger={{ log_level }}, authorizerAppender
+log4j.logger.kafka.authorizer.logger=DEBUG, authorizerInfoAppender, authorizerDebugAppender
 log4j.additivity.kafka.authorizer.logger=false
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/18d7abd8/tests/kafkatest/services/verifiable_producer.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
index 62c4002..945f54b 100644
--- a/tests/kafkatest/services/verifiable_producer.py
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -44,7 +44,6 @@ class VerifiableProducer(BackgroundThreadService):
 
     def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000,
version=TRUNK):
         super(VerifiableProducer, self).__init__(context, num_nodes)
-        self.log_level = "TRACE"
 
         self.kafka = kafka
         self.topic = topic

http://git-wip-us.apache.org/repos/asf/kafka/blob/18d7abd8/tests/kafkatest/tests/produce_consume_validate.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/tests/produce_consume_validate.py b/tests/kafkatest/tests/produce_consume_validate.py
index e01c70f..00988da 100644
--- a/tests/kafkatest/tests/produce_consume_validate.py
+++ b/tests/kafkatest/tests/produce_consume_validate.py
@@ -59,14 +59,18 @@ class ProduceConsumeValidateTest(Test):
 
     def run_produce_consume_validate(self, core_test_action=None, *args):
         """Top-level template for simple produce/consume/validate tests."""
+        try:
+            self.start_producer_and_consumer()
 
-        self.start_producer_and_consumer()
+            if core_test_action is not None:
+                core_test_action(*args)
 
-        if core_test_action is not None:
-            core_test_action(*args)
-
-        self.stop_producer_and_consumer()
-        self.validate()
+            self.stop_producer_and_consumer()
+            self.validate()
+        except BaseException as e:
+            for s in self.test_context.services:
+                self.mark_for_collect(s)
+            raise e
 
     def validate(self):
         """Check that each acked message was consumed."""


Mime
View raw message