public void run()

in pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java [145:357]


    public void run() throws Exception {
        CommandLine commander = spec.commandLine();

        if (this.metadataStoreUrl == null && this.zookeeperServers == null) {
            System.err.println("Metadata store address argument is required (--metadata-store)");
            commander.usage(commander.getOut());
            PerfClientUtils.exit(1);
        }

        // Dump config variables
        PerfClientUtils.printJVMInformation(log);
        ObjectMapper m = new ObjectMapper();
        ObjectWriter w = m.writerWithDefaultPrettyPrinter();
        log.info("Starting Pulsar managed-ledger perf writer with config: {}", w.writeValueAsString(this));

        byte[] payloadData = new byte[this.msgSize];
        ByteBuf payloadBuffer = PulsarByteBufAllocator.DEFAULT.directBuffer(this.msgSize);
        payloadBuffer.writerIndex(this.msgSize);

        // Now processing command line arguments
        String managedLedgerPrefix = "test-" + DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 5);

        if (this.metadataStoreUrl == null) {
            this.metadataStoreUrl = this.zookeeperServers;
        }

        ClientConfiguration bkConf = new ClientConfiguration();
        bkConf.setUseV2WireProtocol(true);
        bkConf.setAddEntryTimeout(30);
        bkConf.setReadEntryTimeout(30);
        bkConf.setThrottleValue(0);
        bkConf.setNumChannelsPerBookie(this.maxConnections);
        bkConf.setMetadataServiceUri(this.metadataStoreUrl);

        ManagedLedgerFactoryConfig mlFactoryConf = new ManagedLedgerFactoryConfig();
        mlFactoryConf.setMaxCacheSize(0);

        @Cleanup
        MetadataStoreExtended metadataStore = MetadataStoreExtended.create(this.metadataStoreUrl,
                MetadataStoreConfig.builder().metadataStoreName(MetadataStoreConfig.METADATA_STORE).build());
        ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkConf, mlFactoryConf);

        ManagedLedgerConfig mlConf = new ManagedLedgerConfig();
        mlConf.setEnsembleSize(this.ensembleSize);
        mlConf.setWriteQuorumSize(this.writeQuorum);
        mlConf.setAckQuorumSize(this.ackQuorum);
        mlConf.setMinimumRolloverTime(10, TimeUnit.MINUTES);
        mlConf.setMetadataEnsembleSize(this.ensembleSize);
        mlConf.setMetadataWriteQuorumSize(this.writeQuorum);
        mlConf.setMetadataAckQuorumSize(this.ackQuorum);
        mlConf.setDigestType(this.digestType);
        mlConf.setMaxSizePerLedgerMb(2048);

        List<CompletableFuture<ManagedLedger>> futures = new ArrayList<>();

        for (int i = 0; i < this.numManagedLedgers; i++) {
            String name = String.format("%s-%03d", managedLedgerPrefix, i);
            CompletableFuture<ManagedLedger> future = new CompletableFuture<>();
            futures.add(future);
            factory.asyncOpen(name, mlConf, new OpenLedgerCallback() {

                @Override
                public void openLedgerComplete(ManagedLedger ledger, Object ctx) {
                    future.complete(ledger);
                }

                @Override
                public void openLedgerFailed(ManagedLedgerException exception, Object ctx) {
                    future.completeExceptionally(exception);
                }
            }, null, null);
        }

        List<ManagedLedger> managedLedgers = futures.stream().map(CompletableFuture::join).collect(Collectors.toList());

        log.info("Created {} managed ledgers", managedLedgers.size());

        long start = System.nanoTime();
        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
            printAggregatedThroughput(start);
            printAggregatedStats();
        }));

        Collections.shuffle(managedLedgers);
        AtomicBoolean isDone = new AtomicBoolean();

        Map<Integer, List<ManagedLedger>> managedLedgersPerThread = allocateToThreads(managedLedgers,
                this.numThreads);

        for (int i = 0; i < this.numThreads; i++) {
            List<ManagedLedger> managedLedgersForThisThread = managedLedgersPerThread.get(i);
            int nunManagedLedgersForThisThread = managedLedgersForThisThread.size();
            long numMessagesForThisThread = this.numMessages / this.numThreads;
            int maxOutstandingForThisThread = this.maxOutstanding;

            executor.submit(() -> {
                try {
                    final double msgRate = this.msgRate / (double) this.numThreads;
                    final RateLimiter rateLimiter = RateLimiter.create(msgRate);

                    // Acquire 1 sec worth of messages to have a slower ramp-up
                    rateLimiter.acquire((int) msgRate);
                    final long startTime = System.nanoTime();
                    final long testEndTime = startTime + (long) (this.testTime * 1e9);

                    final Semaphore semaphore = new Semaphore(maxOutstandingForThisThread);

                    final AddEntryCallback addEntryCallback = new AddEntryCallback() {
                        @Override
                        public void addComplete(Position position, ByteBuf entryData, Object ctx) {
                            long sendTime = (Long) (ctx);
                            messagesSent.increment();
                            bytesSent.add(payloadData.length);
                            totalMessagesSent.increment();
                            totalBytesSent.add(payloadData.length);

                            long latencyMicros = NANOSECONDS.toMicros(System.nanoTime() - sendTime);
                            recorder.recordValue(latencyMicros);
                            cumulativeRecorder.recordValue(latencyMicros);

                            semaphore.release();
                        }

                        @Override
                        public void addFailed(ManagedLedgerException exception, Object ctx) {
                            log.warn("Write error on message", exception);
                            PerfClientUtils.exit(1);
                        }
                    };

                    // Send messages on all topics/producers
                    long totalSent = 0;
                    while (true) {
                        for (int j = 0; j < nunManagedLedgersForThisThread; j++) {
                            if (this.testTime > 0) {
                                if (System.nanoTime() > testEndTime) {
                                    log.info("------------- DONE (reached the maximum duration: [{} seconds] of "
                                            + "production) --------------", this.testTime);
                                    isDone.set(true);
                                    Thread.sleep(5000);
                                    PerfClientUtils.exit(0);
                                }
                            }

                            if (numMessagesForThisThread > 0) {
                                if (totalSent++ >= numMessagesForThisThread) {
                                    log.info("------------- DONE (reached the maximum number: [{}] of production) "
                                            + "--------------", numMessagesForThisThread);
                                    isDone.set(true);
                                    Thread.sleep(5000);
                                    PerfClientUtils.exit(0);
                                }
                            }

                            semaphore.acquire();
                            rateLimiter.acquire();

                            final long sendTime = System.nanoTime();
                            managedLedgersForThisThread.get(j).asyncAddEntry(payloadBuffer, addEntryCallback, sendTime);
                        }
                    }
                } catch (Throwable t) {
                    log.error("Got error", t);
                }
            });
        }

        // Print report stats
        long oldTime = System.nanoTime();

        Histogram reportHistogram = null;

        while (true) {
            try {
                Thread.sleep(10000);
            } catch (InterruptedException e) {
                break;
            }

            if (isDone.get()) {
                break;
            }

            long now = System.nanoTime();
            double elapsed = (now - oldTime) / 1e9;

            long total = totalMessagesSent.sum();
            double rate = messagesSent.sumThenReset() / elapsed;
            double throughput = bytesSent.sumThenReset() / elapsed / 1024 / 1024 * 8;

            reportHistogram = recorder.getIntervalHistogram(reportHistogram);

            log.info(
                    "Throughput produced: {} msg --- {}  msg/s --- {} Mbit/s --- Latency: mean: {} ms - med: {} "
                            + "- 95pct: {} - 99pct: {} - 99.9pct: {} - 99.99pct: {} - Max: {}",
                    INTFORMAT.format(total),
                    THROUGHPUTFORMAT.format(rate),
                    THROUGHPUTFORMAT.format(throughput),
                    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));

            reportHistogram.reset();

            oldTime = now;
        }

        factory.shutdown();
    }