Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
82 changes: 72 additions & 10 deletions tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java
Original file line number Diff line number Diff line change
Expand Up @@ -3,9 +3,9 @@
* 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.
Expand All @@ -14,11 +14,18 @@

import static net.sourceforge.argparse4j.impl.Arguments.store;

import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import java.util.Random;

import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
Expand All @@ -42,15 +49,36 @@ public static void main(String[] args) throws Exception {
/* parse args */
String topicName = res.getString("topic");
long numRecords = res.getLong("numRecords");
int recordSize = res.getInt("recordSize");
Integer recordSize = res.getInt("recordSize");
int throughput = res.getInt("throughput");
List<String> producerProps = res.getList("producerConfig");
String producerConfig = res.getString("producerConfigFile");
String payloadFilePath = res.getString("payloadFile");

// since default value gets printed with the help text, we are escaping \n there and replacing it with correct value here.
String payloadDelimiter = res.getString("payloadDelimiter").equals("\\n") ? "\n" : res.getString("payloadDelimiter");

if (producerProps == null && producerConfig == null) {
throw new ArgumentParserException("Either --producer-props or --producer.config must be specified.", parser);
}

List<byte[]> payloadByteList = new ArrayList<>();
if (payloadFilePath != null) {
Path path = Paths.get(payloadFilePath);
System.out.println("Reading payloads from: " + path.toAbsolutePath());
if (Files.notExists(path) || Files.size(path) == 0) {
throw new IllegalArgumentException("File does not exist or empty file provided.");
}

String[] payloadList = new String(Files.readAllBytes(path), "UTF-8").split(payloadDelimiter);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this seems like it could be dangerous depending on how large your input file is. it would probably be best to use BufferedReader, but if you have to support custom delimiters this could get complicated.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@norwood We have to store all the messages in memory, otherwise, picking a new random message to send will be slow and complicated.
Given that all messages are stored in memory, I think its OK to use Files.readAllBytes()

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@norwood Do let me know your thoughts on the point mentioned above.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry. yeah, this makes sense. thx.

you will still need to get a commiter to 👍 this, but lgtm


System.out.println("Number of messages read: " + payloadList.length);

for (String payload : payloadList) {
payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8));
}
}

Properties props = new Properties();
if (producerConfig != null) {
props.putAll(Utils.loadProps(producerConfig));
Expand All @@ -68,16 +96,24 @@ public static void main(String[] args) throws Exception {
KafkaProducer<byte[], byte[]> producer = new KafkaProducer<byte[], byte[]>(props);

/* setup perf test */
byte[] payload = new byte[recordSize];
byte[] payload = null;
Random random = new Random(0);
for (int i = 0; i < payload.length; ++i)
payload[i] = (byte) (random.nextInt(26) + 65);
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topicName, payload);
if (recordSize != null) {
payload = new byte[recordSize];
for (int i = 0; i < payload.length; ++i)
payload[i] = (byte) (random.nextInt(26) + 65);
}
ProducerRecord<byte[], byte[]> record;
Stats stats = new Stats(numRecords, 5000);
long startMs = System.currentTimeMillis();

ThroughputThrottler throttler = new ThroughputThrottler(throughput, startMs);
for (int i = 0; i < numRecords; i++) {
if (payloadFilePath != null) {
payload = payloadByteList.get(random.nextInt(payloadByteList.size()));
}
record = new ProducerRecord<>(topicName, payload);

long sendStartMs = System.currentTimeMillis();
Callback cb = stats.nextCompletion(sendStartMs, payload.length, stats);
producer.send(record, cb);
Expand Down Expand Up @@ -109,6 +145,11 @@ private static ArgumentParser argParser() {
.defaultHelp(true)
.description("This tool is used to verify the producer performance.");

MutuallyExclusiveGroup payloadOptions = parser
.addMutuallyExclusiveGroup()
.required(true)
.description("either --record-size or --payload-file must be specified but not both.");

parser.addArgument("--topic")
.action(store())
.required(true)
Expand All @@ -124,13 +165,34 @@ private static ArgumentParser argParser() {
.dest("numRecords")
.help("number of messages to produce");

parser.addArgument("--record-size")
payloadOptions.addArgument("--record-size")
.action(store())
.required(true)
.required(false)
.type(Integer.class)
.metavar("RECORD-SIZE")
.dest("recordSize")
.help("message size in bytes");
.help("message size in bytes. Note that you must provide exactly one of --record-size or --payload-file.");

payloadOptions.addArgument("--payload-file")
.action(store())
.required(false)
.type(String.class)
.metavar("PAYLOAD-FILE")
.dest("payloadFile")
.help("file to read the message payloads from. This works only for UTF-8 encoded text files. " +
"Payloads will be read from this file and a payload will be randomly selected when sending messages. " +
"Note that you must provide exactly one of --record-size or --payload-file.");

parser.addArgument("--payload-delimiter")
.action(store())
.required(false)
.type(String.class)
.metavar("PAYLOAD-DELIMITER")
.dest("payloadDelimiter")
.setDefault("\\n")
.help("provides delimiter to be used when --payload-file is provided. " +
"Defaults to new line. " +
"Note that this parameter will be ignored if --payload-file is not provided.");

parser.addArgument("--throughput")
.action(store())
Expand Down