You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/12/06 14:44:29 UTC

[GitHub] [kafka] mimaison commented on a change in pull request #11456: KAFKA-13351: Add possibility to write kafka headers in Kafka Console Producer

mimaison commented on a change in pull request #11456:
URL: https://github.com/apache/kafka/pull/11456#discussion_r763051523



##########
File path: core/src/main/scala/kafka/tools/ConsoleProducer.scala
##########
@@ -206,11 +210,25 @@ object ConsoleProducer {
       .describedAs("size")
       .ofType(classOf[java.lang.Integer])
       .defaultsTo(1024*100)
-    val propertyOpt = parser.accepts("property", "A mechanism to pass user-defined properties in the form key=value to the message reader. " +
-      "This allows custom configuration for a user-defined message reader. Default properties include:\n" +
-      "\tparse.key=true|false\n" +
-      "\tkey.separator=<key.separator>\n" +
-      "\tignore.error=true|false")
+    val propertyOpt = parser.accepts("property",
+      """A mechanism to pass user-defined properties in the form key=value to the message reader. This allows custom configuration for a user-defined message reader.
+        |Default properties include:
+        | parse.key=false
+        | parse.headers=false
+        | ignore.error=false
+        | key.separator=\t
+        | headers.delimiter=\t
+        | headers.separator=,
+        | headers.key.separator=:
+        |Default parsing pattern when:
+        | parse.headers=true & parse.key=true:
+        |  "h1:v1,h2...\tkey\tvalue"

Review comment:
       Should we have `h1:v1,h2:v2...\tkey\tvalue` ?
   Same below

##########
File path: core/src/test/scala/unit/kafka/tools/LineMessageReaderTest.scala
##########
@@ -0,0 +1,193 @@
+/**
+ * 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 unit.kafka.tools
+
+import kafka.tools.ConsoleProducer.LineMessageReader
+import org.apache.kafka.clients.producer.ProducerRecord
+import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.header.Header
+import org.apache.kafka.common.header.internals.RecordHeader
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows}
+import org.junit.jupiter.api.Test
+
+import java.io.ByteArrayInputStream
+import java.lang
+import java.util.Arrays.asList
+import java.util.Properties
+
+class LineMessageReaderTest {
+
+  private def defaultTestProps = {
+    val props = new Properties()
+    props.put("topic", "topic")
+    props.put("parse.key", "true")
+    props.put("parse.headers", "true")
+    props
+  }
+
+  @Test
+  def testLineReader(): Unit = {
+    val input =
+    "key0\tvalue0\n" +
+    "key1\tvalue1"
+
+    val props = defaultTestProps
+    props.put("parse.headers", "false")
+
+    runTest(props, input, record("key0", "value0"), record("key1", "value1"))
+  }
+
+  @Test
+  def testLineReaderHeader(): Unit = {
+
+    val input = "headerKey0:headerValue0,headerKey1:headerValue1\tkey0\tvalue0\n"
+
+    val expectedHeaders: lang.Iterable[Header] = asList(
+      new RecordHeader("headerKey0", "headerValue0".getBytes()),
+      new RecordHeader("headerKey1", "headerValue1".getBytes())
+    )
+
+    val expected = record("key0", "value0", expectedHeaders)
+
+    runTest(defaultTestProps, input, expected)
+  }
+
+  @Test
+  def testLineReaderHeaderNoKey(): Unit = {
+    val input = "headerKey:headerValue\tvalue\n"
+
+    val props = defaultTestProps
+    props.put("parse.key", "false")
+
+    val expectedHeaders: lang.Iterable[Header] = asList(new RecordHeader("headerKey", "headerValue".getBytes()))
+    runTest(props, input, record(null, "value", expectedHeaders))
+  }
+
+  @Test
+  def testLineReaderOnlyValue(): Unit = {
+    val props = defaultTestProps
+    props.put("parse.key", "false")
+    props.put("parse.headers", "false")
+
+    runTest(props, "value\n", record(null, "value"))
+  }
+
+  @Test
+  def testParseHeaderEnabledWithCustomDelimiterAndVaryingNumberOfKeyValueHeaderPairs(): Unit = {
+    val props = defaultTestProps
+    props.put("key.separator", "#")
+    props.put("parse.headers", "true")
+    props.put("headers.delimiter", "!")
+    props.put("headers.separator", "&")
+    props.put("headers.key.separator", ":")
+
+    val input =
+      "headerKey0.0:headerValue0.0&headerKey0.1:headerValue0.1!key0#value0\n" +
+        "headerKey1.0:headerValue1.0!key1#value1"
+
+    val headers: lang.Iterable[Header] = asList(
+      new RecordHeader("headerKey0.0", "headerValue0.0".getBytes()),
+      new RecordHeader("headerKey0.1", "headerValue0.1".getBytes())
+    )
+
+    val record0 = new ProducerRecord("topic", null, null, "key0", "value0", headers)
+

Review comment:
       We can remove this empty line. Same for the line after `runTest()` below

##########
File path: core/src/main/scala/kafka/tools/ConsoleProducer.scala
##########
@@ -206,11 +210,25 @@ object ConsoleProducer {
       .describedAs("size")
       .ofType(classOf[java.lang.Integer])
       .defaultsTo(1024*100)
-    val propertyOpt = parser.accepts("property", "A mechanism to pass user-defined properties in the form key=value to the message reader. " +
-      "This allows custom configuration for a user-defined message reader. Default properties include:\n" +
-      "\tparse.key=true|false\n" +
-      "\tkey.separator=<key.separator>\n" +
-      "\tignore.error=true|false")
+    val propertyOpt = parser.accepts("property",
+      """A mechanism to pass user-defined properties in the form key=value to the message reader. This allows custom configuration for a user-defined message reader.
+        |Default properties include:
+        | parse.key=false
+        | parse.headers=false
+        | ignore.error=false
+        | key.separator=\t
+        | headers.delimiter=\t
+        | headers.separator=,
+        | headers.key.separator=:
+        |Default parsing pattern when:
+        | parse.headers=true & parse.key=true:

Review comment:
       Should we use `and` rather than `&` ?

##########
File path: core/src/test/scala/unit/kafka/tools/LineMessageReaderTest.scala
##########
@@ -0,0 +1,193 @@
+/**
+ * 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 unit.kafka.tools
+
+import kafka.tools.ConsoleProducer.LineMessageReader
+import org.apache.kafka.clients.producer.ProducerRecord
+import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.header.Header
+import org.apache.kafka.common.header.internals.RecordHeader
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows}
+import org.junit.jupiter.api.Test
+
+import java.io.ByteArrayInputStream
+import java.lang
+import java.util.Arrays.asList
+import java.util.Properties
+
+class LineMessageReaderTest {
+
+  private def defaultTestProps = {
+    val props = new Properties()
+    props.put("topic", "topic")
+    props.put("parse.key", "true")
+    props.put("parse.headers", "true")
+    props
+  }
+
+  @Test
+  def testLineReader(): Unit = {
+    val input =
+    "key0\tvalue0\n" +
+    "key1\tvalue1"
+
+    val props = defaultTestProps
+    props.put("parse.headers", "false")
+
+    runTest(props, input, record("key0", "value0"), record("key1", "value1"))
+  }
+
+  @Test
+  def testLineReaderHeader(): Unit = {
+
+    val input = "headerKey0:headerValue0,headerKey1:headerValue1\tkey0\tvalue0\n"
+
+    val expectedHeaders: lang.Iterable[Header] = asList(
+      new RecordHeader("headerKey0", "headerValue0".getBytes()),
+      new RecordHeader("headerKey1", "headerValue1".getBytes())
+    )
+
+    val expected = record("key0", "value0", expectedHeaders)
+
+    runTest(defaultTestProps, input, expected)
+  }
+
+  @Test
+  def testLineReaderHeaderNoKey(): Unit = {
+    val input = "headerKey:headerValue\tvalue\n"
+
+    val props = defaultTestProps
+    props.put("parse.key", "false")
+
+    val expectedHeaders: lang.Iterable[Header] = asList(new RecordHeader("headerKey", "headerValue".getBytes()))
+    runTest(props, input, record(null, "value", expectedHeaders))
+  }
+
+  @Test
+  def testLineReaderOnlyValue(): Unit = {
+    val props = defaultTestProps
+    props.put("parse.key", "false")
+    props.put("parse.headers", "false")
+
+    runTest(props, "value\n", record(null, "value"))
+  }
+
+  @Test
+  def testParseHeaderEnabledWithCustomDelimiterAndVaryingNumberOfKeyValueHeaderPairs(): Unit = {
+    val props = defaultTestProps
+    props.put("key.separator", "#")
+    props.put("parse.headers", "true")
+    props.put("headers.delimiter", "!")
+    props.put("headers.separator", "&")
+    props.put("headers.key.separator", ":")
+
+    val input =
+      "headerKey0.0:headerValue0.0&headerKey0.1:headerValue0.1!key0#value0\n" +
+        "headerKey1.0:headerValue1.0!key1#value1"

Review comment:
       Can we align it with the line above?
   Same in a few tests below

##########
File path: core/src/test/scala/unit/kafka/tools/LineMessageReaderTest.scala
##########
@@ -0,0 +1,193 @@
+/**
+ * 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 unit.kafka.tools
+
+import kafka.tools.ConsoleProducer.LineMessageReader
+import org.apache.kafka.clients.producer.ProducerRecord
+import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.header.Header
+import org.apache.kafka.common.header.internals.RecordHeader
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows}
+import org.junit.jupiter.api.Test
+
+import java.io.ByteArrayInputStream
+import java.lang
+import java.util.Arrays.asList
+import java.util.Properties
+
+class LineMessageReaderTest {
+
+  private def defaultTestProps = {
+    val props = new Properties()
+    props.put("topic", "topic")
+    props.put("parse.key", "true")
+    props.put("parse.headers", "true")
+    props
+  }
+
+  @Test
+  def testLineReader(): Unit = {
+    val input =
+    "key0\tvalue0\n" +
+    "key1\tvalue1"
+
+    val props = defaultTestProps
+    props.put("parse.headers", "false")
+
+    runTest(props, input, record("key0", "value0"), record("key1", "value1"))
+  }
+
+  @Test
+  def testLineReaderHeader(): Unit = {
+
+    val input = "headerKey0:headerValue0,headerKey1:headerValue1\tkey0\tvalue0\n"
+
+    val expectedHeaders: lang.Iterable[Header] = asList(
+      new RecordHeader("headerKey0", "headerValue0".getBytes()),
+      new RecordHeader("headerKey1", "headerValue1".getBytes())
+    )
+
+    val expected = record("key0", "value0", expectedHeaders)
+
+    runTest(defaultTestProps, input, expected)
+  }
+
+  @Test
+  def testLineReaderHeaderNoKey(): Unit = {
+    val input = "headerKey:headerValue\tvalue\n"
+
+    val props = defaultTestProps
+    props.put("parse.key", "false")
+
+    val expectedHeaders: lang.Iterable[Header] = asList(new RecordHeader("headerKey", "headerValue".getBytes()))
+    runTest(props, input, record(null, "value", expectedHeaders))
+  }
+
+  @Test
+  def testLineReaderOnlyValue(): Unit = {
+    val props = defaultTestProps
+    props.put("parse.key", "false")
+    props.put("parse.headers", "false")
+
+    runTest(props, "value\n", record(null, "value"))
+  }
+
+  @Test
+  def testParseHeaderEnabledWithCustomDelimiterAndVaryingNumberOfKeyValueHeaderPairs(): Unit = {
+    val props = defaultTestProps
+    props.put("key.separator", "#")
+    props.put("parse.headers", "true")
+    props.put("headers.delimiter", "!")
+    props.put("headers.separator", "&")
+    props.put("headers.key.separator", ":")
+
+    val input =
+      "headerKey0.0:headerValue0.0&headerKey0.1:headerValue0.1!key0#value0\n" +
+        "headerKey1.0:headerValue1.0!key1#value1"
+
+    val headers: lang.Iterable[Header] = asList(

Review comment:
       Should this be named `headers0`?

##########
File path: core/src/main/scala/kafka/tools/ConsoleProducer.scala
##########
@@ -264,38 +282,113 @@ object ConsoleProducer {
     var reader: BufferedReader = null
     var parseKey = false
     var keySeparator = "\t"
+    var parseHeader = false
+    var headersDelimiter = "\t"
+    var headersSeparator = ","
+    var headerKeySeparator = ":"
     var ignoreError = false
     var lineNumber = 0
     var printPrompt = System.console != null
+    var headerSeparatorPattern: Pattern = _
+    var headerKeySeparatorPattern: Pattern = _
 
     override def init(inputStream: InputStream, props: Properties): Unit = {
       topic = props.getProperty("topic")
       if (props.containsKey("parse.key"))
         parseKey = props.getProperty("parse.key").trim.equalsIgnoreCase("true")
       if (props.containsKey("key.separator"))
         keySeparator = props.getProperty("key.separator")
+      if (props.containsKey("parse.headers"))
+        parseHeader = props.getProperty("parse.headers").trim.equalsIgnoreCase("true")
+      if (props.containsKey("headers.delimiter"))
+        headersDelimiter = props.getProperty("headers.delimiter")
+      if (props.containsKey("headers.separator"))
+        headersSeparator = props.getProperty("headers.separator")
+        headerSeparatorPattern = Pattern.compile(headersSeparator)
+      if (props.containsKey("headers.key.separator"))
+        headerKeySeparator = props.getProperty("headers.key.separator")
+        headerKeySeparatorPattern = Pattern.compile(headerKeySeparator)
       if (props.containsKey("ignore.error"))
         ignoreError = props.getProperty("ignore.error").trim.equalsIgnoreCase("true")
       reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))
     }
 
-    override def readMessage() = {
+    override def readMessage(): ProducerRecord[Array[Byte],Array[Byte]] = {
       lineNumber += 1
-      if (printPrompt)
-        print(">")
-      (reader.readLine(), parseKey) match {
-        case (null, _) => null
-        case (line, true) =>
-          line.indexOf(keySeparator) match {
-            case -1 =>
-              if (ignoreError) new ProducerRecord(topic, line.getBytes(StandardCharsets.UTF_8))
-              else throw new KafkaException(s"No key found on line $lineNumber: $line")
-            case n =>
-              val value = (if (n + keySeparator.size > line.size) "" else line.substring(n + keySeparator.size)).getBytes(StandardCharsets.UTF_8)
-              new ProducerRecord(topic, line.substring(0, n).getBytes(StandardCharsets.UTF_8), value)
-          }
-        case (line, false) =>
-          new ProducerRecord(topic, line.getBytes(StandardCharsets.UTF_8))
+      if (printPrompt) print(">")
+      val line = reader.readLine()
+      try{
+        (line, parseKey, parseHeader) match {
+          case (null, _, _) => null
+          case (line, true, true) => parseHeaderKeyValue(line)
+          case (line, true, false) => parseKeyValue(line)
+          case (line, false, true) => parseHeaderValue(line)
+          case (line, false, false) => parseValue(line)
+        }
+      } catch {
+        case _: Throwable => onMatchError(line)
+      }
+    }
+
+    private def parseHeaderKeyValue(line: String): ProducerRecord[Array[Byte],Array[Byte]] = {
+      val headerIndex = line.indexOf(headersDelimiter)
+      val headers = line.substring(0, headerIndex)
+      val keyValue = line.substring(headerIndex + 1)
+      val keyIndex = keyValue.indexOf(keySeparator)
+      val key = keyValue.substring(0, keyIndex)
+      val value = keyValue.substring(keyIndex + 1)
+      new ProducerRecord(topic, null, null, key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8), mapHeaders(headers))
+    }
+
+    private def parseKeyValue(line: String): ProducerRecord[Array[Byte],Array[Byte]] = {
+      val keyIndex = line.indexOf(keySeparator)
+      val key = line.substring(0, keyIndex)
+      val value = line.substring(keyIndex + 1)
+      new ProducerRecord(topic, key.getBytes(StandardCharsets.UTF_8), value.getBytes(StandardCharsets.UTF_8))
+    }
+
+    private def parseHeaderValue(line: String): ProducerRecord[Array[Byte],Array[Byte]] = {
+      val headerIndex = line.indexOf(headersDelimiter)
+      val headers = line.substring(0, headerIndex)
+      val value = line.substring(headerIndex + 1)
+      new ProducerRecord(topic, null, null, null, value.getBytes(StandardCharsets.UTF_8), mapHeaders(headers))
+    }
+
+    private def parseValue(line: String): ProducerRecord[Array[Byte],Array[Byte]] = {
+      new ProducerRecord(topic, line.getBytes(StandardCharsets.UTF_8))
+    }
+
+    private def onMatchError(line: String): ProducerRecord[Array[Byte], Array[Byte]] = {
+      if (ignoreError) {
+        new ProducerRecord(topic, line.getBytes(StandardCharsets.UTF_8))
+      } else {
+        throw new KafkaException("Could not parse line " + lineNumber + ", most likely line does not match pattern: " + illustratePattern())

Review comment:
       We can use string interpolation here: `KafkaException(s"Could not parse line $lineNumber ...` rather than concatenation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org