You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ew...@apache.org on 2016/08/31 04:31:24 UTC

kafka git commit: KAFKA-4062: Make --print-data-log implicit if --offsets-decoder is enabled for DumpLogSegments

Repository: kafka
Updated Branches:
  refs/heads/trunk b91eeac94 -> 2f20a3987


KAFKA-4062: Make --print-data-log implicit if --offsets-decoder is enabled for DumpLogSegments

set print-data-log option when offset-decoder is set.  hachikuji we had talked about this one before, does this change look ok to you?

Author: Dustin Cote <du...@confluent.io>

Reviewers: Jason Gustafson <ja...@confluent.io>, Ewen Cheslack-Postava <ew...@confluent.io>

Closes #1797 from cotedm/KAFKA-4062


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

Branch: refs/heads/trunk
Commit: 2f20a398735f9c0d39487bedc2d5dd7f4d3eb179
Parents: b91eeac
Author: Dustin Cote <du...@confluent.io>
Authored: Tue Aug 30 21:31:09 2016 -0700
Committer: Ewen Cheslack-Postava <me...@ewencp.org>
Committed: Tue Aug 30 21:31:09 2016 -0700

----------------------------------------------------------------------
 .../scala/kafka/tools/DumpLogSegments.scala     | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/2f20a398/core/src/main/scala/kafka/tools/DumpLogSegments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
index 0a659f4..fb46fa7 100755
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -37,11 +37,11 @@ object DumpLogSegments {
 
   def main(args: Array[String]) {
     val parser = new OptionParser
-    val printOpt = parser.accepts("print-data-log", "if set, printing the messages content when dumping data logs")
-    val verifyOpt = parser.accepts("verify-index-only", "if set, just verify the index log without printing its content")
+    val printOpt = parser.accepts("print-data-log", "if set, printing the messages content when dumping data logs. Automatically set if any decoder option is specified.")
+    val verifyOpt = parser.accepts("verify-index-only", "if set, just verify the index log without printing its content.")
     val indexSanityOpt = parser.accepts("index-sanity-check", "if set, just checks the index sanity without printing its content. " +
       "This is the same check that is executed on broker startup to determine if an index needs rebuilding or not.")
-    val filesOpt = parser.accepts("files", "REQUIRED: The comma separated list of data and index log files to be dumped")
+    val filesOpt = parser.accepts("files", "REQUIRED: The comma separated list of data and index log files to be dumped.")
                            .withRequiredArg
                            .describedAs("file1, file2, ...")
                            .ofType(classOf[String])
@@ -50,7 +50,7 @@ object DumpLogSegments {
                                   .describedAs("size")
                                   .ofType(classOf[java.lang.Integer])
                                   .defaultsTo(5 * 1024 * 1024)
-    val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration")
+    val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration.")
     val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.")
                                .withOptionalArg()
                                .ofType(classOf[java.lang.String])
@@ -59,7 +59,7 @@ object DumpLogSegments {
                                .withOptionalArg()
                                .ofType(classOf[java.lang.String])
                                .defaultsTo("kafka.serializer.StringDecoder")
-    val offsetsOpt = parser.accepts("offsets-decoder", "if set, log data will be parsed as offset data from __consumer_offsets topic")
+    val offsetsOpt = parser.accepts("offsets-decoder", "if set, log data will be parsed as offset data from __consumer_offsets topic.")
 
 
     if(args.length == 0)
@@ -69,13 +69,13 @@ object DumpLogSegments {
 
     CommandLineUtils.checkRequiredArgs(parser, options, filesOpt)
 
-    val print = if(options.has(printOpt)) true else false
-    val verifyOnly = if(options.has(verifyOpt)) true else false
-    val indexSanityOnly = if(options.has(indexSanityOpt)) true else false
+    val printDataLog = (options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt))
+    val verifyOnly = options.has(verifyOpt)
+    val indexSanityOnly = options.has(indexSanityOpt)
 
     val files = options.valueOf(filesOpt).split(",")
     val maxMessageSize = options.valueOf(maxMessageSizeOpt).intValue()
-    val isDeepIteration = if(options.has(deepIterationOpt)) true else false
+    val isDeepIteration = options.has(deepIterationOpt)
 
     val messageParser = if (options.has(offsetsOpt)) {
       new OffsetsMessageParser
@@ -93,7 +93,7 @@ object DumpLogSegments {
       val file = new File(arg)
       if(file.getName.endsWith(Log.LogFileSuffix)) {
         println("Dumping " + file)
-        dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize , messageParser)
+        dumpLog(file, printDataLog, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize , messageParser)
       } else if(file.getName.endsWith(Log.IndexFileSuffix)) {
         println("Dumping " + file)
         dumpIndex(file, indexSanityOnly, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize)