in pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java [172:506]
public void run() throws Exception {
super.parseCLI();
// Dump config variables
PerfClientUtils.printJVMInformation(log);
ObjectMapper m = new ObjectMapper();
ObjectWriter w = m.writerWithDefaultPrettyPrinter();
log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(this));
final byte[] payloadBytes = new byte[1024];
Random random = new Random(0);
for (int i = 0; i < payloadBytes.length; ++i) {
payloadBytes[i] = (byte) (random.nextInt(26) + 65);
}
if (this.partitions != null) {
final PulsarAdminBuilder adminBuilder = PerfClientUtils
.createAdminBuilderFromArguments(this, this.adminURL);
try (PulsarAdmin adminClient = adminBuilder.build()) {
for (String topic : this.producerTopic) {
log.info("Creating produce 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);
}
}
}
}
}
ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(this)
.enableTransaction(!this.isDisableTransaction);
try (PulsarClient client = clientBuilder.build()) {
ExecutorService executorService = new ThreadPoolExecutor(this.numTestThreads,
this.numTestThreads,
0L, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<>());
long startTime = System.nanoTime();
long testEndTime = startTime + (long) (this.testTime * 1e9);
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
if (!this.isDisableTransaction) {
printTxnAggregatedThroughput(startTime);
} else {
printAggregatedThroughput(startTime);
}
printAggregatedStats();
}));
// start perf test
AtomicBoolean executing = new AtomicBoolean(true);
RateLimiter rateLimiter = this.openTxnRate > 0
? RateLimiter.create(this.openTxnRate)
: null;
for (int i = 0; i < this.numTestThreads; i++) {
executorService.submit(() -> {
//The producer and consumer clients are built in advance, and then this thread is
//responsible for the production and consumption tasks of the transaction through the loop.
//A thread may perform tasks of multiple transactions in a traversing manner.
List<Producer<byte[]>> producers = null;
List<List<Consumer<byte[]>>> consumers = null;
AtomicReference<Transaction> atomicReference = null;
try {
producers = buildProducers(client);
consumers = buildConsumer(client);
if (!this.isDisableTransaction) {
atomicReference = new AtomicReference<>(client.newTransaction()
.withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS)
.build()
.get());
} else {
atomicReference = new AtomicReference<>(null);
}
} catch (Exception e) {
log.error("Failed to build Producer/Consumer with exception : ", e);
executorService.shutdownNow();
PerfClientUtils.exit(1);
}
//The while loop has no break, and finally ends the execution through the shutdownNow of
//the executorService
while (true) {
if (this.numTransactions > 0) {
if (totalNumTxnOpenTxnFail.sum()
+ totalNumTxnOpenTxnSuccess.sum() >= this.numTransactions) {
if (totalNumEndTxnOpFailed.sum()
+ totalNumEndTxnOpSuccess.sum() < this.numTransactions) {
continue;
}
log.info("------------------- DONE -----------------------");
executing.compareAndSet(true, false);
executorService.shutdownNow();
PerfClientUtils.exit(0);
break;
}
}
if (this.testTime > 0) {
if (System.nanoTime() > testEndTime) {
log.info("------------------- DONE -----------------------");
executing.compareAndSet(true, false);
executorService.shutdownNow();
PerfClientUtils.exit(0);
break;
}
}
Transaction transaction = atomicReference.get();
for (List<Consumer<byte[]>> subscriptions : consumers) {
for (Consumer<byte[]> consumer : subscriptions) {
for (int j = 0; j < this.numMessagesReceivedPerTransaction; j++) {
Message<byte[]> message = null;
try {
message = consumer.receive();
} catch (PulsarClientException e) {
log.error("Receive message failed", e);
executorService.shutdownNow();
PerfClientUtils.exit(1);
}
long receiveTime = System.nanoTime();
if (!this.isDisableTransaction) {
consumer.acknowledgeAsync(message.getMessageId(), transaction)
.thenRun(() -> {
long latencyMicros = NANOSECONDS.toMicros(
System.nanoTime() - receiveTime);
messageAckRecorder.recordValue(latencyMicros);
messageAckCumulativeRecorder.recordValue(latencyMicros);
numMessagesAckSuccess.increment();
}).exceptionally(exception -> {
if (exception instanceof InterruptedException && !executing.get()) {
return null;
}
log.error(
"Ack message failed with transaction {} throw exception",
transaction, exception);
numMessagesAckFailed.increment();
return null;
});
} else {
consumer.acknowledgeAsync(message).thenRun(() -> {
long latencyMicros = NANOSECONDS.toMicros(
System.nanoTime() - receiveTime);
messageAckRecorder.recordValue(latencyMicros);
messageAckCumulativeRecorder.recordValue(latencyMicros);
numMessagesAckSuccess.increment();
}).exceptionally(exception -> {
if (exception instanceof InterruptedException && !executing.get()) {
return null;
}
log.error(
"Ack message failed with transaction {} throw exception",
transaction, exception);
numMessagesAckFailed.increment();
return null;
});
}
}
}
}
for (Producer<byte[]> producer : producers) {
for (int j = 0; j < this.numMessagesProducedPerTransaction; j++) {
long sendTime = System.nanoTime();
if (!this.isDisableTransaction) {
producer.newMessage(transaction).value(payloadBytes)
.sendAsync().thenRun(() -> {
long latencyMicros = NANOSECONDS.toMicros(
System.nanoTime() - sendTime);
messageSendRecorder.recordValue(latencyMicros);
messageSendRCumulativeRecorder.recordValue(latencyMicros);
numMessagesSendSuccess.increment();
}).exceptionally(exception -> {
if (exception instanceof InterruptedException && !executing.get()) {
return null;
}
log.error("Send transaction message failed with exception : ",
exception);
numMessagesSendFailed.increment();
return null;
});
} else {
producer.newMessage().value(payloadBytes)
.sendAsync().thenRun(() -> {
long latencyMicros = NANOSECONDS.toMicros(
System.nanoTime() - sendTime);
messageSendRecorder.recordValue(latencyMicros);
messageSendRCumulativeRecorder.recordValue(latencyMicros);
numMessagesSendSuccess.increment();
}).exceptionally(exception -> {
if (exception instanceof InterruptedException && !executing.get()) {
return null;
}
log.error("Send message failed with exception : ", exception);
numMessagesSendFailed.increment();
return null;
});
}
}
}
if (rateLimiter != null) {
rateLimiter.tryAcquire();
}
if (!this.isDisableTransaction) {
if (!this.isAbortTransaction) {
transaction.commit()
.thenRun(() -> {
numTxnOpSuccess.increment();
totalNumEndTxnOpSuccess.increment();
}).exceptionally(exception -> {
if (exception instanceof InterruptedException && !executing.get()) {
return null;
}
log.error("Commit transaction {} failed with exception",
transaction.getTxnID().toString(),
exception);
totalNumEndTxnOpFailed.increment();
return null;
});
} else {
transaction.abort().thenRun(() -> {
numTxnOpSuccess.increment();
totalNumEndTxnOpSuccess.increment();
}).exceptionally(exception -> {
if (exception instanceof InterruptedException && !executing.get()) {
return null;
}
log.error("Commit transaction {} failed with exception",
transaction.getTxnID().toString(),
exception);
totalNumEndTxnOpFailed.increment();
return null;
});
}
while (true) {
try {
Transaction newTransaction = client.newTransaction()
.withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS)
.build()
.get();
atomicReference.compareAndSet(transaction, newTransaction);
totalNumTxnOpenTxnSuccess.increment();
break;
} catch (Exception throwable) {
if (throwable instanceof InterruptedException && !executing.get()) {
break;
}
log.error("Failed to new transaction with exception: ", throwable);
totalNumTxnOpenTxnFail.increment();
}
}
} else {
totalNumTxnOpenTxnSuccess.increment();
totalNumEndTxnOpSuccess.increment();
numTxnOpSuccess.increment();
}
}
});
}
// Print report stats
long oldTime = System.nanoTime();
Histogram reportSendHistogram = null;
Histogram reportAckHistogram = null;
String statsFileName = "perf-transaction-" + System.currentTimeMillis() + ".hgrm";
log.info("Dumping latency stats to {}", statsFileName);
PrintStream histogramLog = new PrintStream(new FileOutputStream(statsFileName), false);
HistogramLogWriter histogramLogWriter = new HistogramLogWriter(histogramLog);
// Some log header bits
histogramLogWriter.outputLogFormatVersion();
histogramLogWriter.outputLegend();
while (executing.get()) {
try {
Thread.sleep(10000);
} catch (InterruptedException e) {
break;
}
long now = System.nanoTime();
double elapsed = (now - oldTime) / 1e9;
long total = totalNumEndTxnOpFailed.sum() + totalNumTxnOpenTxnSuccess.sum();
double rate = numTxnOpSuccess.sumThenReset() / elapsed;
reportSendHistogram = messageSendRecorder.getIntervalHistogram(reportSendHistogram);
reportAckHistogram = messageAckRecorder.getIntervalHistogram(reportAckHistogram);
String txnOrTaskLog = !this.isDisableTransaction
? "Throughput transaction: {} transaction executes --- {} transaction/s"
: "Throughput task: {} task executes --- {} task/s";
log.info(
txnOrTaskLog + " --- send Latency: mean: {} ms - med: {} "
+ "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}"
+ " --- ack Latency: "
+ "mean: {} ms - med: {} - 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: "
+ "{}",
INTFORMAT.format(total),
DEC.format(rate),
DEC.format(reportSendHistogram.getMean() / 1000.0),
DEC.format(reportSendHistogram.getValueAtPercentile(50) / 1000.0),
DEC.format(reportSendHistogram.getValueAtPercentile(95) / 1000.0),
DEC.format(reportSendHistogram.getValueAtPercentile(99) / 1000.0),
DEC.format(reportSendHistogram.getValueAtPercentile(99.9) / 1000.0),
DEC.format(reportSendHistogram.getValueAtPercentile(99.99) / 1000.0),
DEC.format(reportSendHistogram.getMaxValue() / 1000.0),
DEC.format(reportAckHistogram.getMean() / 1000.0),
DEC.format(reportAckHistogram.getValueAtPercentile(50) / 1000.0),
DEC.format(reportAckHistogram.getValueAtPercentile(95) / 1000.0),
DEC.format(reportAckHistogram.getValueAtPercentile(99) / 1000.0),
DEC.format(reportAckHistogram.getValueAtPercentile(99.9) / 1000.0),
DEC.format(reportAckHistogram.getValueAtPercentile(99.99) / 1000.0),
DEC.format(reportAckHistogram.getMaxValue() / 1000.0));
histogramLogWriter.outputIntervalHistogram(reportSendHistogram);
histogramLogWriter.outputIntervalHistogram(reportAckHistogram);
reportSendHistogram.reset();
reportAckHistogram.reset();
oldTime = now;
}
}
}