in pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java [253:415]
public void run() throws Exception {
// Dump config variables
PerfClientUtils.printJVMInformation(log);
ObjectMapper m = new ObjectMapper();
ObjectWriter w = m.writerWithDefaultPrettyPrinter();
log.info("Starting Pulsar perf producer with config: {}", w.writeValueAsString(this));
// Read payload data from file if needed
final byte[] payloadBytes = new byte[msgSize];
Random random = new Random(0);
List<byte[]> payloadByteList = new ArrayList<>();
if (this.payloadFilename != null) {
Path payloadFilePath = Paths.get(this.payloadFilename);
if (Files.notExists(payloadFilePath) || Files.size(payloadFilePath) == 0) {
throw new IllegalArgumentException("Payload file doesn't exist or it is empty.");
}
// here escaping the default payload delimiter to correct value
String delimiter = this.payloadDelimiter.equals("\\n") ? "\n" : this.payloadDelimiter;
String[] payloadList = new String(Files.readAllBytes(payloadFilePath),
StandardCharsets.UTF_8).split(delimiter);
log.info("Reading payloads from {} and {} records read", payloadFilePath.toAbsolutePath(),
payloadList.length);
for (String payload : payloadList) {
payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8));
}
if (this.formatPayload) {
messageFormatter = getMessageFormatter(this.formatterClass);
}
} else {
for (int i = 0; i < payloadBytes.length; ++i) {
payloadBytes[i] = (byte) (random.nextInt(26) + 65);
}
}
long start = System.nanoTime();
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
executorShutdownNow();
printAggregatedThroughput(start);
printAggregatedStats();
}));
if (this.partitions != null) {
final PulsarAdminBuilder adminBuilder = PerfClientUtils
.createAdminBuilderFromArguments(this, this.adminURL);
try (PulsarAdmin adminClient = adminBuilder.build()) {
for (String topic : this.topics) {
log.info("Creating partitioned topic {} with {} partitions", topic, this.partitions);
try {
adminClient.topics().createPartitionedTopic(topic, this.partitions);
} catch (PulsarAdminException.ConflictException alreadyExists) {
if (log.isDebugEnabled()) {
log.debug("Topic {} already exists: {}", topic, alreadyExists);
}
PartitionedTopicMetadata partitionedTopicMetadata = adminClient.topics()
.getPartitionedTopicMetadata(topic);
if (partitionedTopicMetadata.partitions != this.partitions) {
log.error("Topic {} already exists but it has a wrong number of partitions: {}, "
+ "expecting {}",
topic, partitionedTopicMetadata.partitions, this.partitions);
PerfClientUtils.exit(1);
}
}
}
}
}
CountDownLatch doneLatch = new CountDownLatch(this.numTestThreads);
final long numMessagesPerThread = this.numMessages / this.numTestThreads;
final int msgRatePerThread = this.msgRate / this.numTestThreads;
for (int i = 0; i < this.numTestThreads; i++) {
final int threadIdx = i;
executor.submit(() -> {
log.info("Started performance test thread {}", threadIdx);
runProducer(
threadIdx,
this,
numMessagesPerThread,
msgRatePerThread,
payloadByteList,
payloadBytes,
doneLatch
);
});
}
// Print report stats
long oldTime = System.nanoTime();
Histogram reportHistogram = null;
HistogramLogWriter histogramLogWriter = null;
if (this.histogramFile != null) {
String statsFileName = this.histogramFile;
log.info("Dumping latency stats to {}", statsFileName);
PrintStream histogramLog = new PrintStream(new FileOutputStream(statsFileName), false);
histogramLogWriter = new HistogramLogWriter(histogramLog);
// Some log header bits
histogramLogWriter.outputLogFormatVersion();
histogramLogWriter.outputLegend();
}
while (true) {
try {
Thread.sleep(10000);
} catch (InterruptedException e) {
break;
}
if (doneLatch.getCount() <= 0) {
break;
}
long now = System.nanoTime();
double elapsed = (now - oldTime) / 1e9;
long total = totalMessagesSent.sum();
long totalTxnOpSuccess = 0;
long totalTxnOpFail = 0;
double rateOpenTxn = 0;
double rate = messagesSent.sumThenReset() / elapsed;
double failureRate = messagesFailed.sumThenReset() / elapsed;
double throughput = bytesSent.sumThenReset() / elapsed / 1024 / 1024 * 8;
reportHistogram = recorder.getIntervalHistogram(reportHistogram);
if (this.isEnableTransaction) {
totalTxnOpSuccess = totalEndTxnOpSuccessNum.sum();
totalTxnOpFail = totalEndTxnOpFailNum.sum();
rateOpenTxn = numTxnOpSuccess.sumThenReset() / elapsed;
log.info("--- Transaction : {} transaction end successfully --- {} transaction end failed "
+ "--- {} Txn/s",
totalTxnOpSuccess, totalTxnOpFail, TOTALFORMAT.format(rateOpenTxn));
}
log.info(
"Throughput produced: {} msg --- {} msg/s --- {} Mbit/s --- failure {} msg/s "
+ "--- Latency: mean: "
+ "{} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
INTFORMAT.format(total),
THROUGHPUTFORMAT.format(rate), THROUGHPUTFORMAT.format(throughput),
THROUGHPUTFORMAT.format(failureRate),
DEC.format(reportHistogram.getMean() / 1000.0),
DEC.format(reportHistogram.getValueAtPercentile(50) / 1000.0),
DEC.format(reportHistogram.getValueAtPercentile(95) / 1000.0),
DEC.format(reportHistogram.getValueAtPercentile(99) / 1000.0),
DEC.format(reportHistogram.getValueAtPercentile(99.9) / 1000.0),
DEC.format(reportHistogram.getValueAtPercentile(99.99) / 1000.0),
DEC.format(reportHistogram.getMaxValue() / 1000.0));
if (histogramLogWriter != null) {
histogramLogWriter.outputIntervalHistogram(reportHistogram);
}
reportHistogram.reset();
oldTime = now;
}
}