mimaison commented on a change in pull request #11456: URL: https://github.com/apache/kafka/pull/11456#discussion_r784199669
########## File path: core/src/test/scala/unit/kafka/tools/LineMessageReaderTest.scala ########## @@ -0,0 +1,210 @@ +/** + * 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.junit.jupiter.api.Assertions.{assertEquals, assertThrows} +import org.junit.jupiter.api.Test + +import java.io.ByteArrayInputStream +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\nkey1\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 expected = record("key0", "value0", List("headerKey0" -> "headerValue0", "headerKey1" -> "headerValue1")) + runTest(defaultTestProps, input, expected) + } + + @Test + def minimalValidInputWithHeaderKeyAndValue(): Unit = { + runTest(defaultTestProps, ":\t\t", record("", "", List("" -> ""))) + } + + @Test + def testLineReaderHeaderNoKey(): Unit = { + val input = "headerKey:headerValue\tvalue\n" + + val props = defaultTestProps + props.put("parse.key", "false") + + runTest(props, input, record(null, "value", List("headerKey" -> "headerValue"))) + } + + @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 record0 = record("key0", "value0", List("headerKey0.0" -> "headerValue0.0", "headerKey0.1" -> "headerValue0.1")) + val record1 = record("key1", "value1", List("headerKey1.0" -> "headerValue1.0")) + + runTest(props, input, record0, record1) + } + + @Test + def testMissingHeaderSeparator(): Unit = { Review comment: Shouldn't it be `testMissingKeySeparator()` instead? ########## File path: core/src/main/scala/kafka/tools/ConsoleProducer.scala ########## @@ -264,39 +278,92 @@ 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 = _ 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") if (props.containsKey("ignore.error")) ignoreError = props.getProperty("ignore.error").trim.equalsIgnoreCase("true") + if (headersDelimiter.equals(headersSeparator)) + throw new KafkaException("headers.delimiter and headers.separator may not be equal") + if (headersDelimiter.equals(headerKeySeparator)) + throw new KafkaException("headers.delimiter and headers.key.separator may not be equal") + if (headersSeparator.equals(headerKeySeparator)) + throw new KafkaException("headers.separator and headers.key.separator may not be equal") 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) + if (printPrompt) print(">") + val line = reader.readLine() + line match { + case null => null + case line => + + val headers = parse(parseHeader, line, headersDelimiter, "headers delimiter") + val key = parse(parseKey, line.substring(offset(headers)), keySeparator, "key separator") + val value = line.substring(offset(headers) + offset(key)) + + val record = new ProducerRecord[Array[Byte], Array[Byte]]( + topic, + if (key != null) key.getBytes(StandardCharsets.UTF_8) else null, + if (value != null) value.getBytes(StandardCharsets.UTF_8) else null, + ) + + if (headers != null){ Review comment: nit: Let's add a space before the opening bracket ########## File path: core/src/main/scala/kafka/tools/ConsoleProducer.scala ########## @@ -264,39 +278,92 @@ 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 = _ 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") if (props.containsKey("ignore.error")) ignoreError = props.getProperty("ignore.error").trim.equalsIgnoreCase("true") + if (headersDelimiter.equals(headersSeparator)) + throw new KafkaException("headers.delimiter and headers.separator may not be equal") + if (headersDelimiter.equals(headerKeySeparator)) + throw new KafkaException("headers.delimiter and headers.key.separator may not be equal") + if (headersSeparator.equals(headerKeySeparator)) + throw new KafkaException("headers.separator and headers.key.separator may not be equal") 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) + if (printPrompt) print(">") + val line = reader.readLine() + line match { + case null => null + case line => + + val headers = parse(parseHeader, line, headersDelimiter, "headers delimiter") + val key = parse(parseKey, line.substring(offset(headers)), keySeparator, "key separator") + val value = line.substring(offset(headers) + offset(key)) + + val record = new ProducerRecord[Array[Byte], Array[Byte]]( + topic, + if (key != null) key.getBytes(StandardCharsets.UTF_8) else null, + if (value != null) value.getBytes(StandardCharsets.UTF_8) else null, + ) + + if (headers != null){ + splitHeaders(headers) + .foreach(header => record.headers().add(header._1, header._2)) } - case (line, false) => - new ProducerRecord(topic, line.getBytes(StandardCharsets.UTF_8)) + + record } } + + private def parse(enabled: Boolean, toParse: String, demarcation: String, demarcationName: String): String = { + (enabled, toParse.indexOf(demarcation)) match { + case (false, _) => null + case (_, -1) => + if (ignoreError) null + else throw new KafkaException(s"No $demarcationName found in '$toParse' on line number $lineNumber") + case (_, index) => toParse.substring(0, index) + } + } + + private def splitHeaders(headers: String): Array[(String, Array[Byte])] = { + headerSeparatorPattern.split(headers) + .map(pair => + (pair.indexOf(headerKeySeparator), ignoreError) match { + case (-1, false) => throw new KafkaException(s"No header key separator found in pair '$pair' on line number $lineNumber") + case (-1, true) => (pair, null) + case (i, _) => (pair.substring(0, i), pair.substring(i + 1).getBytes(StandardCharsets.UTF_8)) + }) + } + + private def offset(headers: String) = { + if (headers == null) 0 else headers.length + 1 + } } } + Review comment: These newlines can be deleted -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
