diff --git a/.github/workflows/ci-go-functions.yaml b/.github/workflows/ci-go-functions.yaml index bd8fe29f5e972..08c287b04b557 100644 --- a/.github/workflows/ci-go-functions.yaml +++ b/.github/workflows/ci-go-functions.yaml @@ -76,7 +76,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - go-version: [1.15, 1.16, 1.17] + go-version: ['1.21'] steps: - name: Check out code into the Go module directory @@ -94,7 +94,7 @@ jobs: - name: InstallTool run: | cd pulsar-function-go - wget -O - -q https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s v1.18.0 + wget -O - -q https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s v1.55.2 ./bin/golangci-lint --version - name: Build diff --git a/bouncy-castle/bc/LICENSE b/bouncy-castle/bc/LICENSE index dae8f16df5b82..14f4e76e921d3 100644 --- a/bouncy-castle/bc/LICENSE +++ b/bouncy-castle/bc/LICENSE @@ -205,6 +205,6 @@ This projects includes binary packages with the following licenses: Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar diff --git a/conf/broker.conf b/conf/broker.conf index 808c4e7ff4ee0..4c57fe4c73d8a 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1148,6 +1148,16 @@ managedLedgerCacheEvictionTimeThresholdMillis=1000 # and thus should be set as inactive. managedLedgerCursorBackloggedThreshold=1000 +# Minimum cursors that must be in backlog state to cache and reuse the read entries. +# (Default =0 to disable backlog reach cache) +managedLedgerMinimumBacklogCursorsForCaching=0 + +# Minimum backlog entries for any cursor before start caching reads. +managedLedgerMinimumBacklogEntriesForCaching=1000 + +# Maximum backlog entry difference to prevent caching entries that can't be reused. +managedLedgerMaxBacklogBetweenCursorsForCaching=1000 + # Rate limit the amount of writes per second generated by consumer acking the messages managedLedgerDefaultMarkDeleteRateLimit=1.0 @@ -1850,4 +1860,4 @@ brokerInterceptorsDirectory=./interceptors brokerInterceptors= # Enable or disable the broker interceptor, which is only used for testing for now -disableBrokerInterceptors=true \ No newline at end of file +disableBrokerInterceptors=true diff --git a/conf/proxy.conf b/conf/proxy.conf index 8285e1cb75320..5a9d433f39ceb 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -376,5 +376,7 @@ zooKeeperCacheExpirySeconds=-1 enableProxyStatsEndpoints=true # Whether the '/metrics' endpoint requires authentication. Defaults to true authenticateMetricsEndpoint=true -# Enable cache metrics data, default value is false -metricsBufferResponse=false +# Time in milliseconds that metrics endpoint would time out. Default is 30s. +# Set it to 0 to disable timeout. +metricsServletTimeoutMs=30000 + diff --git a/conf/standalone.conf b/conf/standalone.conf index 5abffb6e68ee3..707ad06a7b01a 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -813,10 +813,10 @@ managedLedgerNewEntriesCheckDelayInMillis=10 managedLedgerMinimumBacklogCursorsForCaching=0 # Minimum backlog entries for any cursor before start caching reads. -managedLedgerMinimumBacklogEntriesForCaching=100 +managedLedgerMinimumBacklogEntriesForCaching=1000 # Maximum backlog entry difference to prevent caching entries that can't be reused. -managedLedgerMaxBacklogBetweenCursorsForCaching=10000 +managedLedgerMaxBacklogBetweenCursorsForCaching=1000 # Use Open Range-Set to cache unacked messages managedLedgerUnackedRangesOpenCacheSetEnabled=true diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c71dc60541ac7..f3881ce74cc45 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -345,34 +345,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.4.jar - - org.apache.bookkeeper-circe-checksum-4.16.4.jar - - org.apache.bookkeeper-cpu-affinity-4.16.4.jar - - org.apache.bookkeeper-statelib-4.16.4.jar - - org.apache.bookkeeper-stream-storage-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-common-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.4.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.4.jar - - org.apache.bookkeeper-stream-storage-server-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.4.jar - - org.apache.bookkeeper.http-http-server-4.16.4.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.4.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.4.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.4.jar - - org.apache.distributedlog-distributedlog-common-4.16.4.jar - - org.apache.distributedlog-distributedlog-core-4.16.4-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.4.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.4.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.4.jar - - org.apache.bookkeeper-native-io-4.16.4.jar + - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar + - org.apache.bookkeeper-circe-checksum-4.16.5.jar + - org.apache.bookkeeper-cpu-affinity-4.16.5.jar + - org.apache.bookkeeper-statelib-4.16.5.jar + - org.apache.bookkeeper-stream-storage-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-common-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar + - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar + - org.apache.bookkeeper-stream-storage-server-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar + - org.apache.bookkeeper.http-http-server-4.16.5.jar + - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar + - org.apache.distributedlog-distributedlog-common-4.16.5.jar + - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar + - org.apache.distributedlog-distributedlog-core-4.16.5.jar + - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar + - org.apache.bookkeeper-native-io-4.16.5.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar @@ -579,10 +579,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcutil-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcutil-jdk18on-1.78.jar Datastax - com.datastax.oss-managed-ledger-3.1.3.2-SNAPSHOT.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index df04985a3a880..96d515c2da165 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -386,9 +386,9 @@ The Apache Software License, Version 2.0 - log4j-web-2.18.0.jar * BookKeeper - - bookkeeper-common-allocator-4.16.4.jar - - cpu-affinity-4.16.4.jar - - circe-checksum-4.16.4.jar + - bookkeeper-common-allocator-4.16.5.jar + - cpu-affinity-4.16.5.jar + - circe-checksum-4.16.5.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient @@ -466,10 +466,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.75.jar - - bcprov-ext-jdk18on-1.75.jar - - bcprov-jdk18on-1.75.jar - - bcutil-jdk18on-1.75.jar + - bcpkix-jdk18on-1.78.jar + - bcprov-ext-jdk18on-1.78.jar + - bcprov-jdk18on-1.78.jar + - bcutil-jdk18on-1.78.jar ------------------------ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f67f534f86d45..ccea125ef83f6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -823,6 +823,11 @@ public void readEntryComplete(Entry entry, Object ctx) { result.entry = entry; counter.countDown(); } + + @Override + public String toString() { + return String.format("Cursor [%s] get Nth entry", ManagedCursorImpl.this); + } }, null); counter.await(ledger.getConfig().getMetadataOperationsTimeoutSeconds(), TimeUnit.SECONDS); @@ -980,13 +985,18 @@ private void checkForNewEntries(OpReadEntry op, ReadEntriesCallback callback, Ob log.debug("[{}] [{}] Re-trying the read at position {}", ledger.getName(), name, op.readPosition); } + if (isClosed()) { + callback.readEntriesFailed(new CursorAlreadyClosedException("Cursor was already closed"), ctx); + return; + } + if (!hasMoreEntries()) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Still no entries available. Register for notification", ledger.getName(), name); } // Let the managed ledger know we want to be notified whenever a new entry is published - ledger.waitingCursors.add(this); + ledger.addWaitingCursor(this); } else { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Skip notification registering since we do have entries available", @@ -1111,6 +1121,9 @@ public long getNumberOfEntriesInBacklog(boolean isPrecise) { messagesConsumedCounter, markDeletePosition, readPosition); } if (isPrecise) { + if (markDeletePosition.compareTo(ledger.getLastPosition()) >= 0) { + return 0; + } return getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition())); } @@ -1265,12 +1278,13 @@ protected void internalResetCursor(PositionImpl proposedReadPosition, if (proposedReadPosition.equals(PositionImpl.EARLIEST)) { newReadPosition = ledger.getFirstPosition(); } else if (proposedReadPosition.equals(PositionImpl.LATEST)) { - newReadPosition = ledger.getLastPosition().getNext(); + newReadPosition = ledger.getNextValidPosition(ledger.getLastPosition()); } else { newReadPosition = proposedReadPosition; } - log.info("[{}] Initiate reset readPosition to {} on cursor {}", ledger.getName(), newReadPosition, name); + log.info("[{}] Initiate reset readPosition from {} to {} on cursor {}", ledger.getName(), readPosition, + newReadPosition, name); synchronized (pendingMarkDeleteOps) { if (!RESET_CURSOR_IN_PROGRESS_UPDATER.compareAndSet(this, FALSE, TRUE)) { @@ -1536,6 +1550,11 @@ public synchronized void readEntryFailed(ManagedLedgerException mle, Object ctx) callback.readEntriesFailed(exception.get(), ctx); } } + + @Override + public String toString() { + return String.format("Cursor [%s] async replay entries", ManagedCursorImpl.this); + } }; positions.stream().filter(position -> !alreadyAcknowledgedPositions.contains(position)) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 42a5216b3ce36..805958531403f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -737,6 +737,7 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo pbInfo, Stat stat) ledgerInfo.ledgerId = pbLedgerInfo.getLedgerId(); ledgerInfo.entries = pbLedgerInfo.hasEntries() ? pbLedgerInfo.getEntries() : null; ledgerInfo.size = pbLedgerInfo.hasSize() ? pbLedgerInfo.getSize() : null; + ledgerInfo.timestamp = pbLedgerInfo.hasTimestamp() ? pbLedgerInfo.getTimestamp() : null; ledgerInfo.isOffloaded = pbLedgerInfo.hasOffloadContext(); if (pbLedgerInfo.hasOffloadContext()) { MLDataFormats.OffloadContext offloadContext = pbLedgerInfo.getOffloadContext(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 10405134be939..32ac345629d32 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -138,6 +138,7 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.LazyLoadableValue; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; import org.apache.pulsar.metadata.api.Stat; import org.slf4j.Logger; @@ -528,7 +529,8 @@ public void operationFailed(MetaStoreException e) { return; } - log.info("[{}] Created ledger {}", name, lh.getId()); + log.info("[{}] Created ledger {} after closed {}", name, lh.getId(), + currentLedger == null ? "null" : currentLedger.getId()); STATE_UPDATER.set(this, State.LedgerOpened); updateLastLedgerCreatedTimeAndScheduleRolloverTask(); currentLedger = lh; @@ -1032,6 +1034,7 @@ public synchronized void asyncDeleteCursor(final String consumerName, final Dele + consumerName), ctx); return; } else if (!cursor.isDurable()) { + cursor.setState(ManagedCursorImpl.State.Closed); cursors.removeCursor(consumerName); deactivateCursorByName(consumerName); callback.deleteCursorComplete(ctx); @@ -1262,6 +1265,12 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { log.error("Error read entry for position {}", nextPos, exception); future.completeExceptionally(exception); } + + @Override + public String toString() { + return String.format("ML [%s] get earliest message publish time of pos", + ManagedLedgerImpl.this.name); + } }, null); return future; @@ -1748,10 +1757,7 @@ synchronized void ledgerClosed(final LedgerHandle lh) { maybeOffloadInBackground(NULL_OFFLOAD_PROMISE); - if (!pendingAddEntries.isEmpty()) { - // Need to create a new ledger to write pending entries - createLedgerAfterClosed(); - } + createLedgerAfterClosed(); } @Override @@ -1763,7 +1769,8 @@ public void skipNonRecoverableLedger(long ledgerId){ synchronized void createLedgerAfterClosed() { if (isNeededCreateNewLedgerAfterCloseLedger()) { - log.info("[{}] Creating a new ledger after closed", name); + log.info("[{}] Creating a new ledger after closed {}", name, + currentLedger == null ? "null" : currentLedger.getId()); STATE_UPDATER.set(this, State.CreatingLedger); this.lastLedgerCreationInitiationTimestamp = System.currentTimeMillis(); mbean.startDataLedgerCreateOp(); @@ -1805,7 +1812,6 @@ public void closeComplete(int rc, LedgerHandle lh, Object o) { } ledgerClosed(lh); - createLedgerAfterClosed(); } }, null); } @@ -2570,15 +2576,13 @@ private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInS } } - private boolean hasLedgerRetentionExpired(long ledgerTimestamp) { - return config.getRetentionTimeMillis() >= 0 - && clock.millis() - ledgerTimestamp > config.getRetentionTimeMillis(); + private boolean hasLedgerRetentionExpired(long retentionTimeMs, long ledgerTimestamp) { + return retentionTimeMs >= 0 && clock.millis() - ledgerTimestamp > retentionTimeMs; } - private boolean isLedgerRetentionOverSizeQuota(long sizeToDelete) { + private boolean isLedgerRetentionOverSizeQuota(long retentionSizeInMB, long totalSizeOfML, long sizeToDelete) { // Handle the -1 size limit as "infinite" size quota - return config.getRetentionSizeInMB() >= 0 - && TOTAL_SIZE_UPDATER.get(this) - sizeToDelete >= config.getRetentionSizeInMB() * MegaByte; + return retentionSizeInMB >= 0 && totalSizeOfML - sizeToDelete >= retentionSizeInMB * MegaByte; } boolean isOffloadedNeedsDelete(OffloadContext offload, Optional offloadPolicies) { @@ -2675,6 +2679,11 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } long slowestReaderLedgerId = -1; + final LazyLoadableValue slowestNonDurationLedgerId = + new LazyLoadableValue(() -> getTheSlowestNonDurationReadPosition().getLedgerId()); + final long retentionSizeInMB = config.getRetentionSizeInMB(); + final long retentionTimeMs = config.getRetentionTimeMillis(); + final long totalSizeOfML = TOTAL_SIZE_UPDATER.get(this); if (!cursors.hasDurableCursors()) { // At this point the lastLedger will be pointing to the // ledger that has just been closed, therefore the +1 to @@ -2683,7 +2692,16 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } else { PositionImpl slowestReaderPosition = cursors.getSlowestReaderPosition(); if (slowestReaderPosition != null) { - slowestReaderLedgerId = slowestReaderPosition.getLedgerId(); + // The slowest reader position is the mark delete position. + // If the slowest reader position point the last entry in the ledger x, + // the slowestReaderLedgerId should be x + 1 and the ledger x could be deleted. + LedgerInfo ledgerInfo = ledgers.get(slowestReaderPosition.getLedgerId()); + if (ledgerInfo != null && ledgerInfo.getLedgerId() != currentLedger.getId() + && ledgerInfo.getEntries() == slowestReaderPosition.getEntryId() + 1) { + slowestReaderLedgerId = slowestReaderPosition.getLedgerId() + 1; + } else { + slowestReaderLedgerId = slowestReaderPosition.getLedgerId(); + } } else { promise.completeExceptionally(new ManagedLedgerException("Couldn't find reader position")); trimmerMutex.unlock(); @@ -2697,7 +2715,10 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { long totalSizeToDelete = 0; // skip ledger if retention constraint met - for (LedgerInfo ls : ledgers.headMap(slowestReaderLedgerId, false).values()) { + Iterator ledgerInfoIterator = + ledgers.headMap(slowestReaderLedgerId, false).values().iterator(); + while (ledgerInfoIterator.hasNext()){ + LedgerInfo ls = ledgerInfoIterator.next(); // currentLedger can not be deleted if (ls.getLedgerId() == currentLedger.getId()) { if (log.isDebugEnabled()) { @@ -2717,8 +2738,9 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } totalSizeToDelete += ls.getSize(); - boolean overRetentionQuota = isLedgerRetentionOverSizeQuota(totalSizeToDelete); - boolean expired = hasLedgerRetentionExpired(ls.getTimestamp()); + boolean overRetentionQuota = isLedgerRetentionOverSizeQuota(retentionSizeInMB, totalSizeOfML, + totalSizeToDelete); + boolean expired = hasLedgerRetentionExpired(retentionTimeMs, ls.getTimestamp()); if (log.isDebugEnabled()) { log.debug( "[{}] Checking ledger {} -- time-old: {} sec -- " @@ -2735,14 +2757,19 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } ledgersToDelete.add(ls); } else { - if (ls.getLedgerId() < getTheSlowestNonDurationReadPosition().getLedgerId()) { - // once retention constraint has been met, skip check - if (log.isDebugEnabled()) { - log.debug("[{}] Ledger {} not deleted. Neither expired nor over-quota", name, - ls.getLedgerId()); - } - invalidateReadHandle(ls.getLedgerId()); + // once retention constraint has been met, skip check + if (log.isDebugEnabled()) { + log.debug("[{}] Ledger {} not deleted. Neither expired nor over-quota", name, ls.getLedgerId()); } + releaseReadHandleIfNoLongerRead(ls.getLedgerId(), slowestNonDurationLedgerId.getValue()); + break; + } + } + + while (ledgerInfoIterator.hasNext()) { + LedgerInfo ls = ledgerInfoIterator.next(); + if (!releaseReadHandleIfNoLongerRead(ls.getLedgerId(), slowestNonDurationLedgerId.getValue())) { + break; } } @@ -2828,6 +2855,21 @@ public void operationFailed(MetaStoreException e) { } } + /** + * @param ledgerId the ledger handle which maybe will be released. + * @return if the ledger handle was released. + */ + private boolean releaseReadHandleIfNoLongerRead(long ledgerId, long slowestNonDurationLedgerId) { + if (ledgerId < slowestNonDurationLedgerId) { + if (log.isDebugEnabled()) { + log.debug("[{}] Ledger {} no longer needs to be read, close the cached readHandle", name, ledgerId); + } + invalidateReadHandle(ledgerId); + return true; + } + return false; + } + protected void doDeleteLedgers(List ledgersToDelete) { PositionImpl currentLastConfirmedEntry = lastConfirmedEntry; // Update metadata @@ -2859,15 +2901,14 @@ void advanceCursorsIfNecessary(List ledgersToDelete) throws LedgerNo return; } - // need to move mark delete for non-durable cursors to the first ledger NOT marked for deletion - // calling getNumberOfEntries latter for a ledger that is already deleted will be problematic and return - // incorrect results - Long firstNonDeletedLedger = ledgers.higherKey(ledgersToDelete.get(ledgersToDelete.size() - 1).getLedgerId()); - if (firstNonDeletedLedger == null) { - throw new LedgerNotExistException("First non deleted Ledger is not found"); + // Just ack messages like a consumer. Normally, consumers will not confirm a position that does not exist, so + // find the latest existing position to ack. + PositionImpl highestPositionToDelete = calculateLastEntryInLedgerList(ledgersToDelete); + if (highestPositionToDelete == null) { + log.warn("[{}] The ledgers to be trim are all empty, skip to advance non-durable cursors: {}", + name, ledgersToDelete); + return; } - PositionImpl highestPositionToDelete = new PositionImpl(firstNonDeletedLedger, -1); - cursors.forEach(cursor -> { // move the mark delete position to the highestPositionToDelete only if it is smaller than the add confirmed // to prevent the edge case where the cursor is caught up to the latest and highestPositionToDelete may be @@ -2891,6 +2932,19 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { }); } + /** + * @return null if all ledgers is empty. + */ + private PositionImpl calculateLastEntryInLedgerList(List ledgersToDelete) { + for (int i = ledgersToDelete.size() - 1; i >= 0; i--) { + LedgerInfo ledgerInfo = ledgersToDelete.get(i); + if (ledgerInfo != null && ledgerInfo.hasEntries() && ledgerInfo.getEntries() > 0) { + return PositionImpl.get(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); + } + } + return null; + } + /** * Delete this ManagedLedger completely from the system. * @@ -3640,23 +3694,30 @@ public PositionImpl getPreviousPosition(PositionImpl position) { * @return true if the position is valid, false otherwise */ public boolean isValidPosition(PositionImpl position) { - PositionImpl last = lastConfirmedEntry; + PositionImpl lac = lastConfirmedEntry; if (log.isDebugEnabled()) { - log.debug("IsValid position: {} -- last: {}", position, last); + log.debug("IsValid position: {} -- last: {}", position, lac); } - if (position.getEntryId() < 0) { + if (!ledgers.containsKey(position.getLedgerId())){ return false; - } else if (position.getLedgerId() > last.getLedgerId()) { + } else if (position.getEntryId() < 0) { return false; - } else if (position.getLedgerId() == last.getLedgerId()) { - return position.getEntryId() <= (last.getEntryId() + 1); + } else if (currentLedger != null && position.getLedgerId() == currentLedger.getId()) { + // If current ledger is empty, the largest read position can be "{current_ledger: 0}". + // Else, the read position can be set to "{LAC + 1}" when subscribe at LATEST, + return (position.getLedgerId() == lac.getLedgerId() && position.getEntryId() <= lac.getEntryId() + 1) + || position.getEntryId() == 0; + } else if (position.getLedgerId() == lac.getLedgerId()) { + // The ledger witch maintains LAC was closed, and there is an empty current ledger. + // If entry id is larger than LAC, it should be "{current_ledger: 0}". + return position.getEntryId() <= lac.getEntryId(); } else { // Look in the ledgers map LedgerInfo ls = ledgers.get(position.getLedgerId()); if (ls == null) { - if (position.getLedgerId() < last.getLedgerId()) { + if (position.getLedgerId() < lac.getLedgerId()) { // Pointing to a non-existing ledger that is older than the current ledger is invalid return false; } else { @@ -3684,7 +3745,11 @@ public PositionImpl getValidPositionAfterSkippedEntries(final PositionImpl posit PositionImpl skippedPosition = position.getPositionAfterEntries(skippedEntryNum); while (!isValidPosition(skippedPosition)) { Long nextLedgerId = ledgers.ceilingKey(skippedPosition.getLedgerId() + 1); + // This means it has jumped to the last position if (nextLedgerId == null) { + if (currentLedgerEntries == 0) { + return PositionImpl.get(currentLedger.getId(), 0); + } return lastConfirmedEntry.getNext(); } skippedPosition = PositionImpl.get(nextLedgerId, 0); @@ -3795,6 +3860,10 @@ public void removeWaitingCursor(ManagedCursor cursor) { this.waitingCursors.remove(cursor); } + public void addWaitingCursor(ManagedCursorImpl cursor) { + this.waitingCursors.add(cursor); + } + public boolean isCursorActive(ManagedCursor cursor) { return activeCursors.get(cursor.getName()) != null; } @@ -4450,9 +4519,10 @@ private void cancelScheduledTasks() { @Override public boolean checkInactiveLedgerAndRollOver() { - long currentTimeMs = System.currentTimeMillis(); - if (currentLedgerEntries > 0 && inactiveLedgerRollOverTimeMs > 0 && currentTimeMs > (lastAddEntryTimeMs - + inactiveLedgerRollOverTimeMs)) { + if (factory.isMetadataServiceAvailable() + && currentLedgerEntries > 0 + && inactiveLedgerRollOverTimeMs > 0 + && System.currentTimeMillis() > (lastAddEntryTimeMs + inactiveLedgerRollOverTimeMs)) { log.info("[{}] Closing inactive ledger, last-add entry {}", name, lastAddEntryTimeMs); if (STATE_UPDATER.compareAndSet(this, State.LedgerOpened, State.ClosingLedger)) { LedgerHandle currentLedger = this.currentLedger; @@ -4471,7 +4541,6 @@ public boolean checkInactiveLedgerAndRollOver() { } ledgerClosed(lh); - createLedgerAfterClosed(); // we do not create ledger here, since topic is inactive for a long time. }, null); return true; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java index 51e56158cad55..77216ce2e4588 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java @@ -70,7 +70,7 @@ public class NonDurableCursorImpl extends ManagedCursorImpl { private void recoverCursor(PositionImpl mdPosition) { Pair lastEntryAndCounter = ledger.getLastPositionAndCounter(); this.readPosition = isReadCompacted() ? mdPosition.getNext() : ledger.getNextValidPosition(mdPosition); - markDeletePosition = mdPosition; + markDeletePosition = ledger.getPreviousPosition(this.readPosition); // Initialize the counter such that the difference between the messages written on the ML and the // messagesConsumed is equal to the current backlog (negated). diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index 7b59c3903d5bc..a79ba3fb5e23b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -209,8 +209,8 @@ public void recycle() { entries = null; nextReadPosition = null; maxPosition = null; - recyclerHandle.recycle(this); skipCondition = null; + recyclerHandle.recycle(this); } private static final Logger log = LoggerFactory.getLogger(OpReadEntry.class); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 944674f6862c2..1fdf69395068f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -143,8 +143,8 @@ public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallba this.getLedgerHandle(position.getLedgerId()) .thenAccept((ledger) -> asyncReadEntry(ledger, position, callback, ctx)) .exceptionally((ex) -> { - log.error("[{}] Error opening ledger for reading at position {} - {}", this.name, position, - ex.getMessage()); + log.error("[{}] Error opening ledger for reading at position {} - {}. Op: {}", this.name, + position, ex.getMessage(), callback); callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(ex.getCause()), ctx); return null; }); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 644f53c3a522d..c9bd64171c15a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -234,15 +234,16 @@ void readTwice() throws Exception { @Test void testPersistentMarkDeleteIfCreateCursorLedgerFailed() throws Exception { - final int entryCount = 10; + final int entryCount = 9; final String cursorName = "c1"; final String mlName = "ml_test"; - final ManagedLedgerConfig mlConfig = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + // Avoid creating new empty ledger after the last ledger is full and remove fail future. + final ManagedLedgerConfig mlConfig = new ManagedLedgerConfig().setMaxEntriesPerLedger(2); ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open(mlName, mlConfig); ManagedCursor cursor = ml.openCursor("c1"); Position lastEntry = null; - for (int i = 0; i < 10; i++) { + for (int i = 0; i < entryCount; i++) { lastEntry = ml.addEntry(("entry-" + i).getBytes(Encoding)); } @@ -809,7 +810,7 @@ void testResetCursor1() throws Exception { assertEquals(firstInNext, cursor.getReadPosition()); moveStatus.set(false); - // reset to a non exist larger ledger should point to the first non-exist entry in the last ledger + // reset to a non exist larger ledger should point to the first non-exist entry in the next ledger PositionImpl latest = new PositionImpl(last.getLedgerId() + 2, 0); try { cursor.resetCursor(latest); @@ -818,11 +819,13 @@ void testResetCursor1() throws Exception { log.warn("error in reset cursor", e.getCause()); } assertTrue(moveStatus.get()); - PositionImpl lastPos = new PositionImpl(last.getLedgerId(), last.getEntryId() + 1); - assertEquals(lastPos, cursor.getReadPosition()); + PositionImpl lastPos = new PositionImpl(last.getLedgerId() + 1, 0); + Awaitility.await().untilAsserted(() -> { + assertEquals(lastPos, cursor.getReadPosition()); + }); moveStatus.set(false); - // reset to latest should point to the first non-exist entry in the last ledger + // reset to latest should point to the first non-exist entry in the next ledger PositionImpl anotherLast = PositionImpl.LATEST; try { cursor.resetCursor(anotherLast); @@ -1701,7 +1704,7 @@ void testMarkDeleteTwice(boolean useOpenRangeSet) throws Exception { @Test(timeOut = 20000, dataProvider = "useOpenRangeSet") void testSkipEntries(boolean useOpenRangeSet) throws Exception { - ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig() + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", new ManagedLedgerConfig() .setUnackedRangesOpenCacheSetEnabled(useOpenRangeSet).setMaxEntriesPerLedger(2)); Position pos; @@ -1715,6 +1718,11 @@ void testSkipEntries(boolean useOpenRangeSet) throws Exception { pos = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); pos = ledger.addEntry("dummy-entry-2".getBytes(Encoding)); + // Wait new empty ledger created completely. + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.ledgers.size(), 2); + }); + // skip entries in same ledger c1.skipEntries(1, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 1); @@ -1722,7 +1730,7 @@ void testSkipEntries(boolean useOpenRangeSet) throws Exception { // skip entries until end of ledger c1.skipEntries(1, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getReadPosition(), pos.getNext()); + assertEquals(c1.getReadPosition(), new PositionImpl(ledger.currentLedger.getId(), 0)); assertEquals(c1.getMarkDeletedPosition(), pos); // skip entries across ledgers @@ -1737,7 +1745,10 @@ void testSkipEntries(boolean useOpenRangeSet) throws Exception { c1.skipEntries(10, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); assertFalse(c1.hasMoreEntries()); - assertEquals(c1.getReadPosition(), pos.getNext()); + // We can not check the ledger id because a cursor leger can be created. + Awaitility.await().untilAsserted(() -> { + assertEquals(c1.getReadPosition().getEntryId(), 0); + }); assertEquals(c1.getMarkDeletedPosition(), pos); } @@ -1759,7 +1770,7 @@ void testSkipEntriesWithIndividualDeletedMessages(boolean useOpenRangeSet) throw c1.skipEntries(3, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getReadPosition(), pos5.getNext()); + assertEquals(c1.getReadPosition(), new PositionImpl(pos5.getLedgerId() + 1, 0)); assertEquals(c1.getMarkDeletedPosition(), pos5); pos1 = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java index 512e90d17f5e8..7b2f8228ad722 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java @@ -381,7 +381,6 @@ public void recoverAfterZnodeVersionError() throws Exception { ledger.addEntry("entry".getBytes()); fail("should fail"); } catch (ManagedLedgerFencedException e) { - assertEquals(e.getCause().getClass(), ManagedLedgerException.BadVersionException.class); // ok } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java index 8695759c99f62..a953b140aba63 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.mledger.impl; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -53,13 +54,17 @@ public void testGetManagedLedgerInfoWithClose() throws Exception { ManagedLedgerInfo info = factory.getManagedLedgerInfo("testGetManagedLedgerInfo"); - assertEquals(info.ledgers.size(), 4); + assertEquals(info.ledgers.size(), 5); assertEquals(info.ledgers.get(0).ledgerId, 3); assertEquals(info.ledgers.get(1).ledgerId, 4); assertEquals(info.ledgers.get(2).ledgerId, 5); assertEquals(info.ledgers.get(3).ledgerId, 6); + for (ManagedLedgerInfo.LedgerInfo linfo : info.ledgers) { + assertNotNull(linfo.timestamp); + } + assertEquals(info.cursors.size(), 1); CursorInfo cursorInfo = info.cursors.get("c1"); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 94692aeea976a..c16c31a7e963e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -36,6 +36,7 @@ import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertSame; @@ -1120,9 +1121,13 @@ public void testTrimmer() throws Exception { cursor.markDelete(lastPosition); - while (ledger.getNumberOfEntries() != 2) { - Thread.sleep(10); - } + Awaitility.await().untilAsserted(() -> { + // The number of entries in the ledger should not contain the entry in the mark delete position. + // last position is the position of entry-3. + // cursor.markDelete(lastPosition); + // only entry-4 is left in the ledger. + assertEquals(ledger.getNumberOfEntries(), 1); + }); } @Test(timeOut = 20000) @@ -2436,7 +2441,7 @@ public void testRetentionSize() throws Exception { Awaitility.await().untilAsserted(() -> { assertTrue(ml.getTotalSize() <= retentionSizeInMB * 1024 * 1024); - assertEquals(ml.getLedgersInfoAsList().size(), 5); + assertEquals(ml.getLedgersInfoAsList().size(), 6); }); } @@ -2636,11 +2641,11 @@ public void testGetNumberOfEntriesInStorage() throws Exception { managedLedger.addEntry(("entry-" + i).getBytes(Encoding)); } - //trigger ledger rollover and wait for the new ledger created - Field stateUpdater = ManagedLedgerImpl.class.getDeclaredField("state"); - stateUpdater.setAccessible(true); - stateUpdater.set(managedLedger, ManagedLedgerImpl.State.LedgerOpened); - managedLedger.rollCurrentLedgerIfFull(); + // trigger ledger rollover and wait for the new ledger created + Awaitility.await().untilAsserted(() -> { + assertEquals("LedgerOpened", WhiteboxImpl.getInternalState(managedLedger, "state").toString()); + }); + managedLedger.createLedgerAfterClosed(); Awaitility.await().untilAsserted(() -> { assertEquals(managedLedger.getLedgersInfo().size(), 3); assertEquals(managedLedger.getState(), ManagedLedgerImpl.State.LedgerOpened); @@ -2694,9 +2699,17 @@ public void testGetNextValidPosition() throws Exception { assertEquals(ledger.getNextValidPosition((PositionImpl) c1.getMarkDeletedPosition()), p1); assertEquals(ledger.getNextValidPosition(p1), p2); - assertEquals(ledger.getNextValidPosition(p3), PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)); - assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)), PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)); - assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId() + 1, p3.getEntryId() + 1)), PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)); + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.getNextValidPosition(p3), PositionImpl.get(p3.getLedgerId() + 1, 0)); + }); + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)), + PositionImpl.get(p3.getLedgerId() + 1, 0)); + }); + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId() + 1, p3.getEntryId() + 1)), + PositionImpl.get(p3.getLedgerId() + 1, 0)); + }); } /** @@ -3035,19 +3048,22 @@ public void testConsumerSubscriptionInitializePosition() throws Exception{ String content = "entry" + i; // 5 bytes ledger.addEntry(content.getBytes()); } + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.currentLedgerSize, 0); + assertEquals(ledger.ledgers.size(), 1); + }); // Open Cursor also adds cursor into activeCursor-container ManagedCursor latestCursor = ledger.openCursor("c1", InitialPosition.Latest); ManagedCursor earliestCursor = ledger.openCursor("c2", InitialPosition.Earliest); // Since getReadPosition returns the next position, we decrease the entryId by 1 - PositionImpl p1 = (PositionImpl) latestCursor.getReadPosition(); PositionImpl p2 = (PositionImpl) earliestCursor.getReadPosition(); Pair latestPositionAndCounter = ledger.getLastPositionAndCounter(); Pair earliestPositionAndCounter = ledger.getFirstPositionAndCounter(); - - assertEquals(latestPositionAndCounter.getLeft().getNext(), p1); - assertEquals(earliestPositionAndCounter.getLeft().getNext(), p2); + // The read position is the valid next position of the last position instead of the next position. + assertEquals(ledger.getNextValidPosition(latestPositionAndCounter.getLeft()), latestCursor.getReadPosition()); + assertEquals(ledger.getNextValidPosition(earliestPositionAndCounter.getLeft()), p2); assertEquals(latestPositionAndCounter.getRight().longValue(), totalInsertedEntries); assertEquals(earliestPositionAndCounter.getRight().longValue(), totalInsertedEntries - earliestCursor.getNumberOfEntriesInBacklog(false)); @@ -3471,7 +3487,8 @@ public void testManagedLedgerRollOverIfFull() throws Exception { ledger.addEntry(new byte[1024 * 1024]); } - Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getLedgersInfoAsList().size(), msgNum / 2)); + Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getLedgersInfoAsList().size(), + msgNum / 2 + 1)); List entries = cursor.readEntries(msgNum); Assert.assertEquals(msgNum, entries.size()); @@ -3486,6 +3503,9 @@ public void testManagedLedgerRollOverIfFull() throws Exception { stateUpdater.setAccessible(true); stateUpdater.set(ledger, ManagedLedgerImpl.State.LedgerOpened); ledger.rollCurrentLedgerIfFull(); + CompletableFuture completableFuture = new CompletableFuture<>(); + ledger.trimConsumedLedgersInBackground(completableFuture); + completableFuture.get(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 1)); Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getTotalSize(), 0)); } @@ -3651,8 +3671,12 @@ public void testInvalidateReadHandleWhenDeleteLedger() throws Exception { } List entryList = cursor.readEntries(3); assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgers.size(), 3); - assertEquals(ledger.ledgerCache.size(), 2); + Awaitility.await().untilAsserted(() -> { + log.error("ledger.ledgerCache.size() : " + ledger.ledgerCache.size()); + assertEquals(ledger.ledgerCache.size(), 3); + assertEquals(ledger.ledgers.size(), 4); + }); + cursor.clearBacklog(); cursor2.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); @@ -3681,15 +3705,15 @@ public void testLockReleaseWhenTrimLedger() throws Exception { } List entryList = cursor.readEntries(entries); assertEquals(entryList.size(), entries); - assertEquals(ledger.ledgers.size(), entries); - assertEquals(ledger.ledgerCache.size(), entries - 1); + assertEquals(ledger.ledgers.size() - 1, entries); + assertEquals(ledger.ledgerCache.size() - 1, entries - 1); cursor.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); // Cleanup fails because ManagedLedgerNotFoundException is thrown Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.ledgers.size(), entries); - assertEquals(ledger.ledgerCache.size(), entries - 1); + assertEquals(ledger.ledgers.size() - 1, entries); + assertEquals(ledger.ledgerCache.size() - 1, entries - 1); }); // The lock is released even if an ManagedLedgerNotFoundException occurs, so it can be called repeatedly Awaitility.await().untilAsserted(() -> @@ -3715,13 +3739,13 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { } List entryList = cursor.readEntries(3); assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgers.size(), 3); - assertEquals(ledger.ledgerCache.size(), 2); + assertEquals(ledger.ledgers.size(), 4); + assertEquals(ledger.ledgerCache.size(), 3); cursor.clearBacklog(); cursor2.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.ledgers.size(), 3); + assertEquals(ledger.ledgers.size(), 4); assertEquals(ledger.ledgerCache.size(), 0); }); @@ -3729,11 +3753,11 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { ManagedCursor cursor3 = ledger.openCursor("test-cursor3", InitialPosition.Earliest); entryList = cursor3.readEntries(3); assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgerCache.size(), 2); + assertEquals(ledger.ledgerCache.size(), 3); cursor3.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.ledgers.size(), 3); + assertEquals(ledger.ledgers.size(), 4); assertEquals(ledger.ledgerCache.size(), 0); }); @@ -3922,6 +3946,30 @@ public void testDontRollOverEmptyInactiveLedgers() throws Exception { ledger.close(); } + @Test + public void testDontRollOverInactiveLedgersWhenMetadataServiceInvalid() throws Exception { + int inactiveLedgerRollOverTimeMs = 5; + @Cleanup("shutdown") + ManagedLedgerFactoryImpl factory = spy(new ManagedLedgerFactoryImpl(metadataStore, bkc)); + // mock metadata service invalid + when(factory.isMetadataServiceAvailable()).thenReturn(false); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setInactiveLedgerRollOverTime(inactiveLedgerRollOverTimeMs, TimeUnit.MILLISECONDS); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("rollover_inactive", config); + + long ledgerId = ledger.currentLedger.getId(); + + Thread.sleep(inactiveLedgerRollOverTimeMs * 5); + ledger.checkInactiveLedgerAndRollOver(); + + Thread.sleep(inactiveLedgerRollOverTimeMs * 5); + ledger.checkInactiveLedgerAndRollOver(); + + assertEquals(ledger.currentLedger.getId(), ledgerId); + + ledger.close(); + } + @Test public void testOffloadTaskCancelled() throws Exception { ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); @@ -4231,4 +4279,45 @@ public void testNoCleanupOffloadLedgerWhenMetadataExceptionHappens() throws Exce verify(ledgerOffloader, times(0)) .deleteOffloaded(eq(ledgerInfo.getLedgerId()), any(), anyMap()); } + + + @DataProvider(name = "closeLedgerByAddEntry") + public Object[][] closeLedgerByAddEntry() { + return new Object[][] {{Boolean.TRUE}, {Boolean.FALSE}}; + } + + @Test(dataProvider = "closeLedgerByAddEntry") + public void testDeleteCurrentLedgerWhenItIsClosed(boolean closeLedgerByAddEntry) throws Exception { + // Setup: Open a manageLedger with one initial entry. + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(10); + ManagedLedgerImpl ml = spy((ManagedLedgerImpl) factory.open("testDeleteCurrentLedgerWhenItIsClosed", + config)); + assertEquals(ml.ledgers.size(), 1); + ml.addEntry(new byte[4]); + // Act: Trigger the rollover of the current ledger. + long currentLedgerID = ml.currentLedger.getId(); + ml.config.setMaximumRolloverTime(10, TimeUnit.MILLISECONDS); + Thread.sleep(10); + if (closeLedgerByAddEntry) { + // Detect the current ledger is full before written entry and close the ledger after writing completely. + ml.addEntry(new byte[4]); + } else { + // Detect the current ledger is full by the timed task. (Imitate: the timed task `checkLedgerRollTask` call + // `rollCurrentLedgerIfFull` periodically). + ml.rollCurrentLedgerIfFull(); + // the ledger closing in the `rollCurrentLedgerIfFull` is async, so the wait is needed. + Awaitility.await().untilAsserted(() -> assertEquals(ml.ledgers.size(), 2)); + } + // Act: Trigger trimming to delete the previous current ledger. + ml.internalTrimLedgers(false, Futures.NULL_PROMISE); + // Verify: A new ledger will be opened after the current ledger is closed and the previous current ledger can be + // deleted. + Awaitility.await().untilAsserted(() -> { + assertEquals(ml.state, ManagedLedgerImpl.State.LedgerOpened); + assertEquals(ml.ledgers.size(), 1); + assertNotEquals(currentLedgerID, ml.currentLedger.getId()); + assertEquals(ml.currentLedgerEntries, 0); + }); + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java index 1e1f7df0a46d5..82141bfd0eeeb 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java @@ -589,12 +589,12 @@ void subscribeToEarliestPositionWithImmediateDeletion() throws Exception { /* Position p1 = */ ledger.addEntry("entry-1".getBytes()); /* Position p2 = */ ledger.addEntry("entry-2".getBytes()); - Position p3 = ledger.addEntry("entry-3".getBytes()); + /* Position p3 = */ ledger.addEntry("entry-3".getBytes()); Thread.sleep(300); ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); - assertEquals(c1.getReadPosition(), p3); - assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(5, -1)); + assertEquals(c1.getReadPosition(), new PositionImpl(6, 0)); + assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(6, -1)); } @Test // (timeOut = 20000) @@ -723,9 +723,10 @@ public void testBacklogStatsWhenDroppingData() throws Exception { CompletableFuture promise = new CompletableFuture<>(); ledger.internalTrimConsumedLedgers(promise); promise.join(); - - assertEquals(nonDurableCursor.getNumberOfEntries(), 6); - assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 6); + // The mark delete position has moved to position 4:1, and the ledger 4 only has one entry, + // so the ledger 4 can be deleted. nonDurableCursor should has the same backlog with durable cursor. + assertEquals(nonDurableCursor.getNumberOfEntries(), 5); + assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 5); c1.close(); ledger.deleteCursor(c1.getName()); @@ -733,8 +734,8 @@ public void testBacklogStatsWhenDroppingData() throws Exception { ledger.internalTrimConsumedLedgers(promise); promise.join(); - assertEquals(nonDurableCursor.getNumberOfEntries(), 1); - assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 1); + assertEquals(nonDurableCursor.getNumberOfEntries(), 0); + assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 0); ledger.close(); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java index 4482e9944c0ce..cc4b3f2481152 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java @@ -148,7 +148,10 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { newPos = sourceML.addEntry(data); // new ledger rolled. newPos = sourceML.addEntry(data); - Awaitility.await().untilAsserted(() -> assertEquals(shadowML.ledgers.size(), 5)); + Awaitility.await().untilAsserted(() -> { + assertEquals(shadowML.ledgers.size(), 6); + assertEquals(shadowML.currentLedgerEntries, 0); + }); assertEquals(future.get(), fakePos); // LCE should be updated. log.info("3.Source.LCE={},Shadow.LCE={}", sourceML.lastConfirmedEntry, shadowML.lastConfirmedEntry); diff --git a/pom.xml b/pom.xml index 80a330f88a6db..4a4bd6429ec59 100644 --- a/pom.xml +++ b/pom.xml @@ -136,7 +136,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.4 + 4.16.5 3.9.2 1.5.0 1.10.0 @@ -155,7 +155,7 @@ flexible messaging model and an intuitive client API. 1.7.32 4.4 2.18.0 - 1.75 + 1.78 1.0.6 1.0.2.4 2.14.2 diff --git a/pulsar-broker-auth-oidc/pom.xml b/pulsar-broker-auth-oidc/pom.xml index b9fe60fbf5419..8aa60ffce0b1e 100644 --- a/pulsar-broker-auth-oidc/pom.xml +++ b/pulsar-broker-auth-oidc/pom.xml @@ -96,6 +96,7 @@ org.bouncycastle + diff --git a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java index 1462b8e293f79..426e575004c98 100644 --- a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java +++ b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java @@ -303,7 +303,8 @@ private CompletableFuture authenticateToken(String token) { return verifyIssuerAndGetJwk(jwt) .thenCompose(jwk -> { try { - if (!jwt.getAlgorithm().equals(jwk.getAlgorithm())) { + // verify the algorithm, if it is set ("alg" is optional in the JWK spec) + if (jwk.getAlgorithm() != null && !jwt.getAlgorithm().equals(jwk.getAlgorithm())) { incrementFailureMetric(AuthenticationExceptionCode.ALGORITHM_MISMATCH); return CompletableFuture.failedFuture( new AuthenticationException("JWK's alg [" + jwk.getAlgorithm() diff --git a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java index d22b70be7f6f1..ab6417667ecb0 100644 --- a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java +++ b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java @@ -75,6 +75,7 @@ public class AuthenticationProviderOpenIDIntegrationTest { // These are the kid values for JWKs in the /keys endpoint String validJwk = "valid"; String invalidJwk = "invalid"; + String validJwkWithoutAlg = "valid_without_alg"; // The valid issuer String issuer; @@ -188,10 +189,16 @@ void beforeClass() throws IOException { "kty":"RSA", "n":"invalid-key", "e":"AQAB" + }, + { + "kid":"%s", + "kty":"RSA", + "n":"%s", + "e":"%s" } ] } - """.formatted(validJwk, n, e, invalidJwk)))); + """.formatted(validJwk, n, e, invalidJwk, validJwkWithoutAlg, n, e)))); server.stubFor( get(urlEqualTo("/missing-kid/.well-known/openid-configuration")) @@ -274,6 +281,14 @@ public void testTokenWithValidJWK() throws Exception { assertEquals(role, provider.authenticateAsync(new AuthenticationDataCommand(token)).get()); } + @Test + public void testTokenWithValidJWKWithoutAlg() throws Exception { + String role = "superuser"; + // test with a key in JWK that does not have an "alg" field. "alg" is optional in the JWK spec + String token = generateToken(validJwkWithoutAlg, issuer, role, "allowed-audience", 0L, 0L, 10000L); + assertEquals(role, provider.authenticateAsync(new AuthenticationDataCommand(token)).get()); + } + @Test public void testTokenWithTrailingSlashAndValidJWK() throws Exception { String role = "superuser"; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index f4337eb18f64c..b833223cb1e3e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -331,6 +331,19 @@ public class ServiceConfiguration implements PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_SERVER, doc = + "Gzip compression is enabled by default. Specific paths can be excluded from compression.\n" + + "There are 2 syntaxes supported, Servlet url-pattern based, and Regex based.\n" + + "If the spec starts with '^' the spec is assumed to be a regex based path spec and will match " + + "with normal Java regex rules.\n" + + "If the spec starts with '/' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for either an exact match or prefix based match.\n" + + "If the spec starts with '*.' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for a suffix based match.\n" + + "All other syntaxes are unsupported.\n" + + "Disable all compression with ^.* or ^.*$") + private List httpServerGzipCompressionExcludedPaths = new ArrayList<>(); + @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the delayed delivery for messages.") private boolean delayedDeliveryEnabled = true; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java index 651abcd8dbbc2..9f15cc33c9bb1 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java @@ -77,7 +77,10 @@ public static void generateSystemMetrics(SimpleTextOutputStream stream, String c } } for (int j = 0; j < sample.labelNames.size(); j++) { - String labelValue = writeEscapedLabelValue(sample.labelValues.get(j)); + String labelValue = sample.labelValues.get(j); + if (labelValue != null && labelValue.indexOf('"') > -1) { + labelValue = labelValue.replace("\"", "\\\""); + } if (j > 0) { stream.write(","); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java index 64d1fcdab6f14..8a41bed29d44f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java @@ -25,9 +25,13 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -35,67 +39,132 @@ import org.slf4j.LoggerFactory; public class PrometheusMetricsServlet extends HttpServlet { - private static final long serialVersionUID = 1L; - private static final int HTTP_STATUS_OK_200 = 200; - private static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; - - private final long metricsServletTimeoutMs; - private final String cluster; + static final int HTTP_STATUS_OK_200 = 200; + static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; + protected final long metricsServletTimeoutMs; + protected final String cluster; protected List metricsProviders; - private ExecutorService executor = null; + protected ExecutorService executor = null; + protected final int executorMaxThreads; public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster) { + this(metricsServletTimeoutMs, cluster, 1); + } + + public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster, int executorMaxThreads) { this.metricsServletTimeoutMs = metricsServletTimeoutMs; this.cluster = cluster; + this.executorMaxThreads = executorMaxThreads; } @Override public void init() throws ServletException { - executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("prometheus-stats")); + if (executorMaxThreads > 0) { + executor = + Executors.newScheduledThreadPool(executorMaxThreads, new DefaultThreadFactory("prometheus-stats")); + } } @Override protected void doGet(HttpServletRequest request, HttpServletResponse response) { AsyncContext context = request.startAsync(); - context.setTimeout(metricsServletTimeoutMs); - executor.execute(() -> { - long start = System.currentTimeMillis(); - HttpServletResponse res = (HttpServletResponse) context.getResponse(); - try { - res.setStatus(HTTP_STATUS_OK_200); - res.setContentType("text/plain;charset=utf-8"); - generateMetrics(cluster, res.getOutputStream()); - } catch (Exception e) { - long end = System.currentTimeMillis(); - long time = end - start; - if (e instanceof EOFException) { - // NO STACKTRACE - log.error("Failed to send metrics, " - + "likely the client or this server closed " - + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); - } else { - log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean taskStarted = new AtomicBoolean(false); + Future future = executor.submit(() -> { + taskStarted.set(true); + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos(metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); } - res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); - } finally { - long end = System.currentTimeMillis(); - long time = end - start; - try { - context.complete(); - } catch (IllegalStateException e) { - // this happens when metricsServletTimeoutMs expires - // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled - log.error("Failed to generate prometheus stats, " - + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + context.complete(); + return; + } + handleAsyncMetricsRequest(context); + }); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); } } + + @Override + public void onTimeout(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + log.warn("Prometheus metrics request timed out"); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + } + + @Override + public void onStartAsync(AsyncEvent asyncEvent) throws IOException { + + } }); + + } + + private void handleAsyncMetricsRequest(AsyncContext context) { + long start = System.currentTimeMillis(); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + try { + generateMetricsSynchronously(res); + } catch (Exception e) { + long end = System.currentTimeMillis(); + long time = end - start; + if (e instanceof EOFException) { + // NO STACKTRACE + log.error("Failed to send metrics, " + + "likely the client or this server closed " + + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); + } else { + log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + } + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + } finally { + long end = System.currentTimeMillis(); + long time = end - start; + try { + context.complete(); + } catch (IllegalStateException e) { + // this happens when metricsServletTimeoutMs expires + // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled + log.error("Failed to generate prometheus stats, " + + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + } + } } - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGeneratorUtils.generate(cluster, outputStream, metricsProviders); + private void generateMetricsSynchronously(HttpServletResponse res) throws IOException { + res.setStatus(HTTP_STATUS_OK_200); + res.setContentType("text/plain;charset=utf-8"); + PrometheusMetricsGeneratorUtils.generate(cluster, res.getOutputStream(), metricsProviders); } @Override diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java new file mode 100644 index 0000000000000..37c9c05e5d53c --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java @@ -0,0 +1,48 @@ +/* + * 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 org.apache.pulsar.broker.web; + +import java.util.List; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; + +public class GzipHandlerUtil { + public static Handler wrapWithGzipHandler(Handler innerHandler, List gzipCompressionExcludedPaths) { + Handler wrappedHandler; + if (isGzipCompressionCompletelyDisabled(gzipCompressionExcludedPaths)) { + // no need to add GZIP handler if it's disabled by setting the excluded path to "^.*" or "^.*$" + wrappedHandler = innerHandler; + } else { + // add GZIP handler which is active when the request contains "Accept-Encoding: gzip" header + GzipHandler gzipHandler = new GzipHandler(); + gzipHandler.setHandler(innerHandler); + if (gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() > 0) { + gzipHandler.setExcludedPaths(gzipCompressionExcludedPaths.toArray(new String[0])); + } + wrappedHandler = gzipHandler; + } + return wrappedHandler; + } + + public static boolean isGzipCompressionCompletelyDisabled(List gzipCompressionExcludedPaths) { + return gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() == 1 + && (gzipCompressionExcludedPaths.get(0).equals("^.*") + || gzipCompressionExcludedPaths.get(0).equals("^.*$")); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 9a2dd5c7c54fd..6bd2bf205620f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -283,6 +283,40 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse }); } + private CompletableFuture revokePermissionsAsync(String topicUri, String role, boolean force) { + return namespaceResources().getPoliciesAsync(namespaceName).thenCompose( + policiesOptional -> { + Policies policies = policiesOptional.orElseThrow(() -> + new RestException(Status.NOT_FOUND, "Namespace does not exist")); + if (!policies.auth_policies.getTopicAuthentication().containsKey(topicUri) + || !policies.auth_policies.getTopicAuthentication().get(topicUri).containsKey(role)) { + log.warn("[{}] Failed to revoke permission from role {} on topic: Not set at topic level {}", + clientAppId(), role, topicUri); + if (force) { + return CompletableFuture.completedFuture(null); + } else { + return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, + "Permissions are not set at the topic level")); + } + } + // Write the new policies to metadata store + return namespaceResources().setPoliciesAsync(namespaceName, p -> { + p.auth_policies.getTopicAuthentication().computeIfPresent(topicUri, (k, roles) -> { + roles.remove(role); + if (roles.isEmpty()) { + return null; + } + return roles; + }); + return p; + }).thenAccept(__ -> + log.info("[{}] Successfully revoke access for role {} - topic {}", clientAppId(), role, + topicUri) + ); + } + ); + } + protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges validateAdminAccessForTenantAsync(namespaceName.getTenant()) @@ -500,7 +534,7 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean protected void internalCreateMissedPartitions(AsyncResponse asyncResponse) { getPartitionedTopicMetadataAsync(topicName, false, false).thenAccept(metadata -> { - if (metadata != null) { + if (metadata != null && metadata.partitions > 0) { CompletableFuture future = validateNamespaceOperationAsync(topicName.getNamespaceObject(), NamespaceOperation.CREATE_TOPIC); future.thenCompose(__ -> tryCreatePartitionsAsync(metadata.partitions)).thenAccept(v -> { @@ -510,6 +544,8 @@ protected void internalCreateMissedPartitions(AsyncResponse asyncResponse) { resumeAsyncResponseExceptionally(asyncResponse, e); return null; }); + } else { + throw new RestException(Status.NOT_FOUND, String.format("Topic %s does not exist", topicName)); } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. @@ -1575,7 +1611,7 @@ protected CompletableFuture internalDeleteSubscriptionAsync(String subName for (int i = 0; i < partitionMetadata.partitions; i++) { TopicName topicNamePartition = topicName.getPartition(i); futures.add(adminClient.topics() - .deleteSubscriptionAsync(topicNamePartition.toString(), subName, false)); + .deleteSubscriptionAsync(topicNamePartition.toString(), subName, force)); } return FutureUtil.waitForAll(futures).handle((result, exception) -> { @@ -1594,8 +1630,7 @@ protected CompletableFuture internalDeleteSubscriptionAsync(String subName return null; }); } - return internalDeleteSubscriptionForNonPartitionedTopicAsync(subName, authoritative, - force); + return internalDeleteSubscriptionForNonPartitionedTopicAsync(subName, authoritative, force); }); } }); @@ -2272,7 +2307,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su .thenCompose(allowAutoTopicCreation -> getPartitionedTopicMetadataAsync(topicName, authoritative, allowAutoTopicCreation).thenAccept(partitionMetadata -> { final int numPartitions = partitionMetadata.partitions; - if (numPartitions > 0) { + if (partitionMetadata.partitions > 0 && !isUnexpectedTopicName(partitionMetadata)) { final CompletableFuture future = new CompletableFuture<>(); final AtomicInteger count = new AtomicInteger(numPartitions); final AtomicInteger failureCount = new AtomicInteger(0); @@ -2746,6 +2781,12 @@ public void readEntryComplete(Entry entry, Object ctx) { } } } + + @Override + public String toString() { + return String.format("Topic [%s] get entry batch size", + PersistentTopicsBase.this.topicName); + } }, null); } catch (NullPointerException npe) { batchSizeFuture.completeExceptionally(new RestException(Status.NOT_FOUND, "Message not found")); @@ -2847,6 +2888,12 @@ public void readEntryComplete(Entry entry, Object ctx) { } } } + + @Override + public String toString() { + return String.format("Topic [%s] internal get message by id", + PersistentTopicsBase.this.topicName); + } }, null); return results; }); @@ -3053,6 +3100,12 @@ public void readEntryComplete(Entry entry, Object ctx) { public void readEntryFailed(ManagedLedgerException exception, Object ctx) { future.completeExceptionally(exception); } + + @Override + public String toString() { + return String.format("Topic [%s] internal examine message async", + PersistentTopicsBase.this.topicName); + } }, null); return future; } catch (ManagedLedgerException exception) { @@ -4107,27 +4160,33 @@ private CompletableFuture internalExpireMessagesNonPartitionedTopicByPosit return; } try { - final MessageExpirer messageExpirer; - if (subName.startsWith(topic.getReplicatorPrefix())) { - String remoteCluster = PersistentReplicator.getRemoteCluster(subName); - messageExpirer = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - } else { - messageExpirer = topic.getSubscription(subName); - } - if (messageExpirer == null) { - final String message = (subName.startsWith(topic.getReplicatorPrefix())) - ? "Replicator not found" : getSubNotFoundErrorMessage(topicName.toString(), subName); - asyncResponse.resume(new RestException(Status.NOT_FOUND, message)); + PersistentSubscription sub = topic.getSubscription(subName); + if (sub == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getSubNotFoundErrorMessage(topicName.toString(), subName))); return; } - CompletableFuture batchSizeFuture = new CompletableFuture<>(); getEntryBatchSize(batchSizeFuture, topic, messageId, batchIndex); batchSizeFuture.thenAccept(bi -> { PositionImpl position = calculatePositionAckSet(isExcluded, bi, batchIndex, messageId); + boolean issued; try { - if (messageExpirer.expireMessages(position)) { + if (subName.startsWith(topic.getReplicatorPrefix())) { + String remoteCluster = PersistentReplicator.getRemoteCluster(subName); + PersistentReplicator repl = (PersistentReplicator) + topic.getPersistentReplicator(remoteCluster); + if (repl == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + "Replicator not found")); + return; + } + issued = repl.expireMessages(position); + } else { + issued = sub.expireMessages(position); + } + if (issued) { log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), position, topicName, subName); } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java index 61f34ef4901ba..9cf861a8e85cf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java @@ -198,12 +198,20 @@ private static boolean isPhysicalNic(Path nicPath) { return false; } // Check the type to make sure it's ethernet (type "1") - String type = readTrimStringFromFile(nicPath.resolve("type")); + final Path nicTypePath = nicPath.resolve("type"); + if (!Files.exists(nicTypePath)) { + if (log.isDebugEnabled()) { + log.debug("Failed to read NIC type, the expected linux type file does not exist." + + " nic_type_path={}", nicTypePath); + } + return false; + } // wireless NICs don't report speed, ignore them. - return Integer.parseInt(type) == ARPHRD_ETHER; - } catch (Exception e) { - log.warn("[LinuxInfo] Failed to read {} NIC type, the detail is: {}", nicPath, e.getMessage()); - // Read type got error. + return Integer.parseInt(readTrimStringFromFile(nicTypePath)) == ARPHRD_ETHER; + } catch (Exception ex) { + if (log.isDebugEnabled()) { + log.debug("Failed to read NIC type. nic_path={}", nicPath, ex); + } return false; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 1471d4a75c175..f66ed2a5c9062 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -1024,7 +1024,6 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, double splitBundleTime = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - startTime)); log.info("Successfully split {} parent namespace-bundle to {} in {} ms", parentBundle, childBundles, splitBundleTime); - namespaceService.onNamespaceBundleSplit(parentBundle); completionFuture.complete(null); }) .exceptionally(ex -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index e04be25fe499c..5236202df0acf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -140,10 +140,6 @@ public class NamespaceService implements AutoCloseable { private final ConcurrentOpenHashMap namespaceClients; private final List bundleOwnershipListeners; - - private final List bundleSplitListeners; - - private final RedirectManager redirectManager; @@ -172,7 +168,6 @@ public NamespaceService(PulsarService pulsar) { this.namespaceClients = ConcurrentOpenHashMap.newBuilder().build(); this.bundleOwnershipListeners = new CopyOnWriteArrayList<>(); - this.bundleSplitListeners = new CopyOnWriteArrayList<>(); this.localBrokerDataCache = pulsar.getLocalMetadataStore().getMetadataCache(LocalBrokerData.class); this.redirectManager = new RedirectManager(pulsar); } @@ -1005,7 +1000,6 @@ void splitAndOwnBundleOnceAndRetry(NamespaceBundle bundle, // affect the split operation which is already safely completed r.forEach(this::unloadNamespaceBundle); } - onNamespaceBundleSplit(bundle); }) .exceptionally(e -> { String msg1 = format( @@ -1246,19 +1240,6 @@ public void onNamespaceBundleUnload(NamespaceBundle bundle) { } } } - - public void onNamespaceBundleSplit(NamespaceBundle bundle) { - for (NamespaceBundleSplitListener bundleSplitListener : bundleSplitListeners) { - try { - if (bundleSplitListener.test(bundle)) { - bundleSplitListener.onSplit(bundle); - } - } catch (Throwable t) { - LOG.error("Call bundle {} split listener {} error", bundle, bundleSplitListener, t); - } - } - } - public void addNamespaceBundleOwnershipListener(NamespaceBundleOwnershipListener... listeners) { Objects.requireNonNull(listeners); for (NamespaceBundleOwnershipListener listener : listeners) { @@ -1269,15 +1250,6 @@ public void addNamespaceBundleOwnershipListener(NamespaceBundleOwnershipListener getOwnedServiceUnits().forEach(bundle -> notifyNamespaceBundleOwnershipListener(bundle, listeners)); } - public void addNamespaceBundleSplitListener(NamespaceBundleSplitListener... listeners) { - Objects.requireNonNull(listeners); - for (NamespaceBundleSplitListener listener : listeners) { - if (listener != null) { - bundleSplitListeners.add(listener); - } - } - } - private void notifyNamespaceBundleOwnershipListener(NamespaceBundle bundle, NamespaceBundleOwnershipListener... listeners) { if (listeners != null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java index 5098890242b6c..310354dcd3b47 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java @@ -166,7 +166,22 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { } if (subscriptionType == SubType.Exclusive && !consumers.isEmpty()) { - return FutureUtil.failedFuture(new ConsumerBusyException("Exclusive consumer is already connected")); + Consumer actConsumer = ACTIVE_CONSUMER_UPDATER.get(this); + if (actConsumer != null) { + return actConsumer.cnx().checkConnectionLiveness().thenCompose(actConsumerStillAlive -> { + if (actConsumerStillAlive == null || actConsumerStillAlive) { + return FutureUtil.failedFuture(new ConsumerBusyException("Exclusive consumer is already" + + " connected")); + } else { + return addConsumer(consumer); + } + }); + } else { + // It should never happen. + + return FutureUtil.failedFuture(new ConsumerBusyException("Active consumer is in a strange state." + + " Active consumer is null, but there are " + consumers.size() + " registered.")); + } } if (subscriptionType == SubType.Failover && isConsumersExceededOnSubscription()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index bc2541c802e63..6ad1697adfc39 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -103,7 +103,10 @@ public void handleExceededBacklogQuota(PersistentTopic persistentTopic, BacklogQ break; case producer_exception: case producer_request_hold: - disconnectProducers(persistentTopic); + if (!advanceSlowestSystemCursor(persistentTopic)) { + // The slowest is not a system cursor. Disconnecting producers to put backpressure. + disconnectProducers(persistentTopic); + } break; default: break; @@ -268,4 +271,27 @@ private void disconnectProducers(PersistentTopic persistentTopic) { }); } + + /** + * Advances the slowest cursor if that is a system cursor. + * + * @param persistentTopic + * @return true if the slowest cursor is a system cursor + */ + private boolean advanceSlowestSystemCursor(PersistentTopic persistentTopic) { + + ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursor slowestConsumer = mLedger.getSlowestConsumer(); + if (slowestConsumer == null) { + return false; + } + + if (PersistentTopic.isDedupCursorName(slowestConsumer.getName())) { + persistentTopic.getMessageDeduplication().takeSnapshot(); + return true; + } + + // We may need to check other system cursors here : replicator, compaction + return false; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 8d289446b7b42..442f14e36576d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -580,8 +580,10 @@ protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpd } protected void startDeduplicationSnapshotMonitor() { + // We do not know whether users will enable deduplication on namespace level/topic level or not, so keep this + // scheduled task runs. int interval = pulsar().getConfiguration().getBrokerDeduplicationSnapshotFrequencyInSeconds(); - if (interval > 0 && pulsar().getConfiguration().isBrokerDeduplicationEnabled()) { + if (interval > 0) { this.deduplicationSnapshotMonitor = OrderedScheduler.newSchedulerBuilder() .name("deduplication-snapshot-monitor") .numThreads(1) @@ -3151,8 +3153,11 @@ public CompletableFuture fetchPartitionedTopicMetadata if (ex.getCause() instanceof MetadataStoreException .AlreadyExistsException) { + log.info("[{}] The partitioned topic is already" + + " created, try to refresh the cache and read" + + " again.", topicName); // The partitioned topic might be created concurrently - fetchPartitionedTopicMetadataAsync(topicName) + fetchPartitionedTopicMetadataAsync(topicName, true) .whenComplete((metadata2, ex2) -> { if (ex2 == null) { future.complete(metadata2); @@ -3161,6 +3166,9 @@ public CompletableFuture fetchPartitionedTopicMetadata } }); } else { + log.error("[{}] operation of creating partitioned" + + " topic metadata failed", + topicName, ex); future.completeExceptionally(ex); } return null; @@ -3206,9 +3214,14 @@ private CompletableFuture createDefaultPartitionedTopi } public CompletableFuture fetchPartitionedTopicMetadataAsync(TopicName topicName) { + return fetchPartitionedTopicMetadataAsync(topicName, false); + } + + public CompletableFuture fetchPartitionedTopicMetadataAsync(TopicName topicName, + boolean refreshCacheAndGet) { // gets the number of partitions from the configuration cache return pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() - .getPartitionedTopicMetadataAsync(topicName).thenApply(metadata -> { + .getPartitionedTopicMetadataAsync(topicName, refreshCacheAndGet).thenApply(metadata -> { // if the partitioned topic is not found in metadata, then the topic is not partitioned return metadata.orElseGet(() -> new PartitionedTopicMetadata()); }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 29ba7cb866ef2..3d0f0ddf60cb7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -1196,10 +1196,20 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { commandSender.sendErrorResponse(requestId, ServerError.ServiceNotReady, "Consumer is already present on the connection"); } else if (existingConsumerFuture.isCompletedExceptionally()){ + log.warn("[{}][{}][{}] A failed consumer with id is already present on the connection," + + " consumerId={}", remoteAddress, topicName, subscriptionName, consumerId); ServerError error = getErrorCodeWithErrorLog(existingConsumerFuture, true, - String.format("Consumer subscribe failure. remoteAddress: %s, subscription: %s", - remoteAddress, subscriptionName)); - consumers.remove(consumerId, existingConsumerFuture); + String.format("A failed consumer with id is already present on the connection." + + " consumerId: %s, remoteAddress: %s, subscription: %s", + consumerId, remoteAddress, subscriptionName)); + /** + * This future may was failed due to the client closed a in-progress subscribing. + * See {@link #handleCloseConsumer(CommandCloseConsumer)} + * Do not remove the failed future at current line, it will be removed after the progress of + * the previous subscribing is done. + * Before the previous subscribing is done, the new subscribe request will always fail. + * This mechanism is in order to prevent more complex logic to handle the race conditions. + */ commandSender.sendErrorResponse(requestId, error, "Consumer that failed is already present on the connection"); } else { @@ -1311,7 +1321,7 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { // Send error back to client, only if not completed already. if (consumerFuture.completeExceptionally(exception)) { commandSender.sendErrorResponse(requestId, - BrokerServiceException.getClientErrorCode(exception), + BrokerServiceException.getClientErrorCode(exception.getCause()), exception.getCause().getMessage()); } consumers.remove(consumerId, consumerFuture); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 71f78e21f938f..4e9e875bcf4c3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -324,34 +324,46 @@ public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle name } } - private @Nonnull CompletableFuture prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) { + @VisibleForTesting + @Nonnull CompletableFuture prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) { requireNonNull(namespace); - return policyCacheInitMap.computeIfAbsent(namespace, (k) -> { - final CompletableFuture> readerCompletableFuture = - createSystemTopicClient(namespace); - readerCaches.put(namespace, readerCompletableFuture); - ownedBundlesCountPerNamespace.putIfAbsent(namespace, new AtomicInteger(1)); - final CompletableFuture initFuture = readerCompletableFuture - .thenCompose(reader -> { - final CompletableFuture stageFuture = new CompletableFuture<>(); - initPolicesCache(reader, stageFuture); - return stageFuture - // Read policies in background - .thenAccept(__ -> readMorePoliciesAsync(reader)); - }); - initFuture.exceptionally(ex -> { - try { - log.error("[{}] Failed to create reader on __change_events topic", namespace, ex); - cleanCacheAndCloseReader(namespace, false); - } catch (Throwable cleanupEx) { - // Adding this catch to avoid break callback chain - log.error("[{}] Failed to cleanup reader on __change_events topic", namespace, cleanupEx); - } - return null; - }); - // let caller know we've got an exception. - return initFuture; - }); + return pulsarService.getPulsarResources().getNamespaceResources().getPoliciesAsync(namespace) + .thenCompose(namespacePolicies -> { + if (namespacePolicies.isEmpty() || namespacePolicies.get().deleted) { + log.info("[{}] skip prepare init policies cache since the namespace is deleted", + namespace); + return CompletableFuture.completedFuture(null); + } + + return policyCacheInitMap.computeIfAbsent(namespace, (k) -> { + final CompletableFuture> readerCompletableFuture = + createSystemTopicClient(namespace); + readerCaches.put(namespace, readerCompletableFuture); + ownedBundlesCountPerNamespace.putIfAbsent(namespace, new AtomicInteger(1)); + final CompletableFuture initFuture = readerCompletableFuture + .thenCompose(reader -> { + final CompletableFuture stageFuture = new CompletableFuture<>(); + initPolicesCache(reader, stageFuture); + return stageFuture + // Read policies in background + .thenAccept(__ -> readMorePoliciesAsync(reader)); + }); + initFuture.exceptionally(ex -> { + try { + log.error("[{}] Failed to create reader on __change_events topic", + namespace, ex); + cleanCacheAndCloseReader(namespace, false); + } catch (Throwable cleanupEx) { + // Adding this catch to avoid break callback chain + log.error("[{}] Failed to cleanup reader on __change_events topic", + namespace, cleanupEx); + } + return null; + }); + // let caller know we've got an exception. + return initFuture; + }); + }); } protected CompletableFuture> createSystemTopicClient( diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index c4319b0f80185..e508661364d74 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -27,6 +27,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenCursorCallback; @@ -132,6 +133,9 @@ public MessageDupUnknownException() { private final String replicatorPrefix; + + private final AtomicBoolean snapshotTaking = new AtomicBoolean(false); + public MessageDeduplication(PulsarService pulsar, PersistentTopic topic, ManagedLedger managedLedger) { this.pulsar = pulsar; this.topic = topic; @@ -153,9 +157,14 @@ private CompletableFuture recoverSequenceIdsMap() { // Replay all the entries and apply all the sequence ids updates log.info("[{}] Replaying {} entries for deduplication", topic.getName(), managedCursor.getNumberOfEntries()); - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); replayCursor(future); - return future; + return future.thenAccept(lastPosition -> { + if (lastPosition != null && snapshotCounter >= snapshotInterval) { + snapshotCounter = 0; + takeSnapshot(lastPosition); + } + }); } /** @@ -164,11 +173,11 @@ private CompletableFuture recoverSequenceIdsMap() { * * @param future future to trigger when the replay is complete */ - private void replayCursor(CompletableFuture future) { + private void replayCursor(CompletableFuture future) { managedCursor.asyncReadEntries(100, new ReadEntriesCallback() { @Override public void readEntriesComplete(List entries, Object ctx) { - + Position lastPosition = null; for (Entry entry : entries) { ByteBuf messageMetadataAndPayload = entry.getDataBuffer(); MessageMetadata md = Commands.parseMessageMetadata(messageMetadataAndPayload); @@ -178,7 +187,8 @@ public void readEntriesComplete(List entries, Object ctx) { highestSequencedPushed.put(producerName, sequenceId); highestSequencedPersisted.put(producerName, sequenceId); producerRemoved(producerName); - + snapshotCounter++; + lastPosition = entry.getPosition(); entry.release(); } @@ -187,7 +197,7 @@ public void readEntriesComplete(List entries, Object ctx) { pulsar.getExecutor().execute(() -> replayCursor(future)); } else { // Done replaying - future.complete(null); + future.complete(lastPosition); } } @@ -432,6 +442,11 @@ private void takeSnapshot(Position position) { if (log.isDebugEnabled()) { log.debug("[{}] Taking snapshot of sequence ids map", topic.getName()); } + + if (!snapshotTaking.compareAndSet(false, true)) { + return; + } + Map snapshot = new TreeMap<>(); highestSequencedPersisted.forEach((producerName, sequenceId) -> { if (snapshot.size() < maxNumberOfProducers) { @@ -446,11 +461,13 @@ public void markDeleteComplete(Object ctx) { log.debug("[{}] Stored new deduplication snapshot at {}", topic.getName(), position); } lastSnapshotTimestamp = System.currentTimeMillis(); + snapshotTaking.set(false); } @Override public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { log.warn("[{}] Failed to store new deduplication snapshot at {}", topic.getName(), position); + snapshotTaking.set(false); } }, null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index 5bf3f5506fa81..6380317724207 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -95,6 +95,14 @@ private void removeFromHashBlocker(long ledgerId, long entryId) { } } + public Long getHash(long ledgerId, long entryId) { + LongPair value = hashesToBeBlocked.get(ledgerId, entryId); + if (value == null) { + return null; + } + return value.first; + } + public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) { if (!allowOutOfOrderDelivery) { List keysToRemove = new ArrayList<>(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index b3d48252efe58..9d0dba798ad88 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -177,9 +177,15 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { } if (isConsumersExceededOnSubscription()) { - log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit", name); + log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit {}", + name, consumer); return FutureUtil.failedFuture(new ConsumerBusyException("Subscription reached max consumers limit")); } + // This is not an expected scenario, it will never happen in expected. Just print a warn log if the unexpected + // scenario happens. See more detail: https://github.com/apache/pulsar/pull/22283. + if (consumerSet.contains(consumer)) { + log.warn("[{}] Attempting to add a consumer that already registered {}", name, consumer); + } consumerList.add(consumer); if (consumerList.size() > 1 @@ -204,15 +210,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE consumerList.remove(consumer); log.info("Removed consumer {} with pending {} acks", consumer, consumer.getPendingAcks().size()); if (consumerList.isEmpty()) { - cancelPendingRead(); - - redeliveryMessages.clear(); - redeliveryTracker.clear(); - if (closeFuture != null) { - log.info("[{}] All consumers removed. Subscription is disconnected", name); - closeFuture.complete(null); - } - totalAvailablePermits = 0; + clearComponentsAfterRemovedAllConsumers(); } else { if (log.isDebugEnabled()) { log.debug("[{}] Consumer are left, reading more entries", name); @@ -229,8 +227,29 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE readMoreEntries(); } } else { - log.info("[{}] Trying to remove a non-connected consumer: {}", name, consumer); + /** + * This is not an expected scenario, it will never happen in expected. + * Just add a defensive code to avoid the topic can not be unloaded anymore: remove the consumers which + * are not mismatch with {@link #consumerSet}. See more detail: https://github.com/apache/pulsar/pull/22270. + */ + log.error("[{}] Trying to remove a non-connected consumer: {}", name, consumer); + consumerList.removeIf(c -> consumer.equals(c)); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } + } + } + + private synchronized void clearComponentsAfterRemovedAllConsumers() { + cancelPendingRead(); + + redeliveryMessages.clear(); + redeliveryTracker.clear(); + if (closeFuture != null) { + log.info("[{}] All consumers removed. Subscription is disconnected", name); + closeFuture.complete(null); } + totalAvailablePermits = 0; } @Override @@ -290,24 +309,25 @@ public synchronized void readMoreEntries() { } NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); - - if (!messagesToReplayNow.isEmpty()) { + NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); + if (!messagesToReplayFiltered.isEmpty()) { if (log.isDebugEnabled()) { - log.debug("[{}] Schedule replay of {} messages for {} consumers", name, messagesToReplayNow.size(), - consumerList.size()); + log.debug("[{}] Schedule replay of {} messages for {} consumers", name, + messagesToReplayFiltered.size(), consumerList.size()); } havePendingReplayRead = true; minReplayedPosition = messagesToReplayNow.first(); Set deletedMessages = topic.isDelayedDeliveryEnabled() - ? asyncReplayEntriesInOrder(messagesToReplayNow) : asyncReplayEntries(messagesToReplayNow); + ? asyncReplayEntriesInOrder(messagesToReplayFiltered) + : asyncReplayEntries(messagesToReplayFiltered); // clear already acked positions from replay bucket deletedMessages.forEach(position -> redeliveryMessages.remove(((PositionImpl) position).getLedgerId(), ((PositionImpl) position).getEntryId())); // if all the entries are acked-entries and cleared up from redeliveryMessages, try to read // next entries as readCompletedEntries-callback was never called - if ((messagesToReplayNow.size() - deletedMessages.size()) == 0) { + if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { havePendingReplayRead = false; readMoreEntriesAsync(); } @@ -316,7 +336,7 @@ public synchronized void readMoreEntries() { log.debug("[{}] Dispatcher read is blocked due to unackMessages {} reached to max {}", name, totalUnackedMessages, topic.getMaxUnackedMessagesOnSubscription()); } - } else if (!havePendingRead) { + } else if (!havePendingRead && hasConsumersNeededNormalRead()) { if (log.isDebugEnabled()) { log.debug("[{}] Schedule read of {} messages for {} consumers", name, messagesToRead, consumerList.size()); @@ -345,7 +365,16 @@ public synchronized void readMoreEntries() { topic.getMaxReadPosition()); } } else { - log.debug("[{}] Cannot schedule next read until previous one is done", name); + if (log.isDebugEnabled()) { + if (!messagesToReplayNow.isEmpty()) { + log.debug("[{}] [{}] Skipping read for the topic: because all entries in replay queue were" + + " filtered out due to the mechanism of Key_Shared mode, and the left consumers have" + + " no permits now", + topic.getName(), getSubscriptionName()); + } else { + log.debug("[{}] Cannot schedule next read until previous one is done", name); + } + } } } else { if (log.isDebugEnabled()) { @@ -506,6 +535,9 @@ public synchronized CompletableFuture disconnectAllConsumers(boolean isRes if (consumerList.isEmpty()) { closeFuture.complete(null); } else { + // Iterator of CopyOnWriteArrayList uses the internal array to do the for-each, and CopyOnWriteArrayList + // will create a new internal array when adding/removing a new item. So remove items in the for-each + // block is safety when the for-each and add/remove are using a same lock. consumerList.forEach(consumer -> consumer.disconnect(isResetCursor)); cancelPendingRead(); } @@ -1084,6 +1116,27 @@ protected synchronized NavigableSet getMessagesToReplayNow(int max } } + /** + * This is a mode method designed for Key_Shared mode. + * Filter out the entries that will be discarded due to the order guarantee mechanism of Key_Shared mode. + * This method is in order to avoid the scenario below: + * - Get positions from the Replay queue. + * - Read entries from BK. + * - The order guarantee mechanism of Key_Shared mode filtered out all the entries. + * - Delivery non entry to the client, but we did a BK read. + */ + protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + return src; + } + + /** + * This is a mode method designed for Key_Shared mode, to avoid unnecessary stuck. + * See detail {@link PersistentStickyKeyDispatcherMultipleConsumers#hasConsumersNeededNormalRead}. + */ + protected boolean hasConsumersNeededNormalRead() { + return true; + } + protected synchronized boolean shouldPauseDeliveryForDelayTracker() { return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index 15f34258768c5..530230eeb5c12 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -75,6 +75,7 @@ public class PersistentDispatcherSingleActiveConsumer extends AbstractDispatcher protected volatile int readBatchSize; protected final Backoff readFailureBackoff; private volatile ScheduledFuture readOnActiveConsumerTask = null; + private final Object lockForReadOnActiveConsumerTask = new Object(); private final RedeliveryTracker redeliveryTracker; @@ -124,18 +125,23 @@ protected void scheduleReadOnActiveConsumer() { return; } - readOnActiveConsumerTask = topic.getBrokerService().executor().schedule(() -> { - if (log.isDebugEnabled()) { - log.debug("[{}] Rewind cursor and read more entries after {} ms delay", name, - serviceConfig.getActiveConsumerFailoverDelayTimeMillis()); + synchronized (lockForReadOnActiveConsumerTask) { + if (readOnActiveConsumerTask != null) { + return; } - Consumer activeConsumer = ACTIVE_CONSUMER_UPDATER.get(this); - cursor.rewind(activeConsumer != null && activeConsumer.readCompacted()); + readOnActiveConsumerTask = topic.getBrokerService().executor().schedule(() -> { + if (log.isDebugEnabled()) { + log.debug("[{}] Rewind cursor and read more entries after {} ms delay", name, + serviceConfig.getActiveConsumerFailoverDelayTimeMillis()); + } + Consumer activeConsumer = ACTIVE_CONSUMER_UPDATER.get(this); + cursor.rewind(activeConsumer != null && activeConsumer.readCompacted()); - notifyActiveConsumerChanged(activeConsumer); - readMoreEntries(activeConsumer); - readOnActiveConsumerTask = null; - }, serviceConfig.getActiveConsumerFailoverDelayTimeMillis(), TimeUnit.MILLISECONDS); + notifyActiveConsumerChanged(activeConsumer); + readMoreEntries(activeConsumer); + readOnActiveConsumerTask = null; + }, serviceConfig.getActiveConsumerFailoverDelayTimeMillis(), TimeUnit.MILLISECONDS); + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index ac391c1050340..d35b726fb6c5e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -24,7 +24,6 @@ import java.util.SortedMap; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.LongAdder; -import javax.annotation.Nullable; import org.apache.bookkeeper.mledger.AsyncCallbacks.FindEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -48,7 +47,6 @@ public class PersistentMessageExpiryMonitor implements FindEntryCallback, MessageExpirer { private final ManagedCursor cursor; private final String subName; - private final PersistentTopic topic; private final String topicName; private final Rate msgExpired; private final LongAdder totalMsgExpired; @@ -62,10 +60,9 @@ public class PersistentMessageExpiryMonitor implements FindEntryCallback, Messag expirationCheckInProgressUpdater = AtomicIntegerFieldUpdater .newUpdater(PersistentMessageExpiryMonitor.class, "expirationCheckInProgress"); - public PersistentMessageExpiryMonitor(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, - @Nullable PersistentSubscription subscription) { - this.topic = topic; - this.topicName = topic.getName(); + public PersistentMessageExpiryMonitor(String topicName, String subscriptionName, ManagedCursor cursor, + PersistentSubscription subscription) { + this.topicName = topicName; this.cursor = cursor; this.subName = subscriptionName; this.subscription = subscription; @@ -121,8 +118,8 @@ private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTL managedLedger.getLedgersInfo().lastKey(), true); MLDataFormats.ManagedLedgerInfo.LedgerInfo info = null; for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) { - if (!ledgerInfo.hasTimestamp() || !MessageImpl.isEntryExpired(messageTTLInSeconds, - ledgerInfo.getTimestamp())) { + if (!ledgerInfo.hasTimestamp() || ledgerInfo.getTimestamp() == 0L + || !MessageImpl.isEntryExpired(messageTTLInSeconds, ledgerInfo.getTimestamp())) { break; } info = ledgerInfo; @@ -141,12 +138,11 @@ private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTL @Override public boolean expireMessages(Position messagePosition) { // If it's beyond last position of this topic, do nothing. - PositionImpl topicLastPosition = (PositionImpl) this.topic.getLastPosition(); - if (topicLastPosition.compareTo((PositionImpl) messagePosition) < 0) { + if (((PositionImpl) subscription.getTopic().getLastPosition()).compareTo((PositionImpl) messagePosition) < 0) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Ignore expire-message scheduled task, given position {} is beyond " - + "current topic's last position {}", topicName, subName, messagePosition, - topicLastPosition); + + "current topic's last position {}", topicName, subName, messagePosition, + subscription.getTopic().getLastPosition()); } return false; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 0d387da1dd912..4b947b64a6ebd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -117,7 +117,7 @@ public PersistentReplicator(String localCluster, PersistentTopic localTopic, Man brokerService, replicationClient); this.topic = localTopic; this.cursor = cursor; - this.expiryMonitor = new PersistentMessageExpiryMonitor(localTopic, + this.expiryMonitor = new PersistentMessageExpiryMonitor(localTopicName, Codec.decode(cursor.getName()), cursor, null); HAVE_PENDING_READ_UPDATER.set(this, FALSE); PENDING_MESSAGES_UPDATER.set(this, 0); @@ -530,6 +530,12 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { public void readEntryComplete(Entry entry, Object ctx) { future.complete(entry); } + + @Override + public String toString() { + return String.format("Replication [%s] peek Nth message", + PersistentReplicator.this.producer.getProducerName()); + } }, null); return future; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 8f05530f58bfa..ee2ebd7ca867e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -30,13 +30,16 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; @@ -165,6 +168,14 @@ protected Map> initialValue() throws Exception { } }; + private static final FastThreadLocal>> localGroupedPositions = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + @Override protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { long totalMessagesSent = 0; @@ -248,15 +259,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis assert consumer != null; // checked when added to groupedEntries List entriesWithSameKey = current.getValue(); int entriesWithSameKeyCount = entriesWithSameKey.size(); - int availablePermits = Math.max(consumer.getAvailablePermits(), 0); - if (consumer.getMaxUnackedMessages() > 0) { - int remainUnAckedMessages = - // Avoid negative number - Math.max(consumer.getMaxUnackedMessages() - consumer.getUnackedMessages(), 0); - availablePermits = Math.min(availablePermits, remainUnAckedMessages); - } - int maxMessagesForC = Math.min(entriesWithSameKeyCount, availablePermits); - int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, entriesWithSameKey, maxMessagesForC, + int availablePermits = getAvailablePermits(consumer); + int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, + entriesWithSameKey.stream().map(Entry::getPosition).collect(Collectors.toList()), availablePermits, readType, consumerStickyKeyHashesMap.get(consumer)); if (log.isDebugEnabled()) { log.debug("[{}] select consumer {} with messages num {}, read type is {}", @@ -289,7 +294,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); totalEntries += filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); - consumer.sendMessages(entriesWithSameKey, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), @@ -332,8 +336,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return false; } - private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, int maxMessages, - ReadType readType, Set stickyKeyHashes) { + private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, + int availablePermits, ReadType readType, Set stickyKeyHashes) { + int maxMessages = Math.min(entries.size(), availablePermits); if (maxMessages == 0) { return 0; } @@ -378,7 +383,7 @@ private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List en // Here, the consumer is one that has recently joined, so we can only send messages that were // published before it has joined. for (int i = 0; i < maxMessages; i++) { - if (((PositionImpl) entries.get(i).getPosition()).compareTo(maxReadPosition) >= 0) { + if (((PositionImpl) entries.get(i)).compareTo(maxReadPosition) >= 0) { // We have already crossed the divider line. All messages in the list are now // newer than what we can currently dispatch to this consumer return i; @@ -405,6 +410,9 @@ && removeConsumersFromRecentJoinedConsumers()) { } private boolean removeConsumersFromRecentJoinedConsumers() { + if (MapUtils.isEmpty(recentlyJoinedConsumers)) { + return false; + } Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; PositionImpl mdp = (PositionImpl) cursor.getMarkDeletedPosition(); @@ -437,6 +445,76 @@ protected synchronized NavigableSet getMessagesToReplayNow(int max } } + private int getAvailablePermits(Consumer c) { + int availablePermits = Math.max(c.getAvailablePermits(), 0); + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + return availablePermits; + } + + @Override + protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + if (src.isEmpty()) { + return src; + } + NavigableSet res = new TreeSet<>(); + // Group positions. + final Map> groupedPositions = localGroupedPositions.get(); + groupedPositions.clear(); + for (PositionImpl pos : src) { + Long stickyKeyHash = redeliveryMessages.getHash(pos.getLedgerId(), pos.getEntryId()); + if (stickyKeyHash == null) { + res.add(pos); + continue; + } + Consumer c = selector.select(stickyKeyHash.intValue()); + if (c == null) { + // Maybe using HashRangeExclusiveStickyKeyConsumerSelector. + continue; + } + groupedPositions.computeIfAbsent(c, k -> new ArrayList<>()).add(pos); + } + // Filter positions by the Recently Joined Position rule. + for (Map.Entry> item : groupedPositions.entrySet()) { + int availablePermits = getAvailablePermits(item.getKey()); + if (availablePermits == 0) { + continue; + } + int posCountToRead = getRestrictedMaxEntriesForConsumer(item.getKey(), item.getValue(), availablePermits, + ReadType.Replay, null); + if (posCountToRead > 0) { + res.addAll(item.getValue().subList(0, posCountToRead)); + } + } + return res; + } + + /** + * In Key_Shared mode, the consumer will not receive any entries from a normal reading if it is included in + * {@link #recentlyJoinedConsumers}, they can only receive entries from replay reads. + * If all entries in {@link #redeliveryMessages} have been filtered out due to the order guarantee mechanism, + * Broker need a normal read to make the consumers not included in @link #recentlyJoinedConsumers} will not be + * stuck. See https://github.com/apache/pulsar/pull/7105. + */ + @Override + protected boolean hasConsumersNeededNormalRead() { + for (Consumer consumer : consumerList) { + if (consumer == null || consumer.isBlocked()) { + continue; + } + if (recentlyJoinedConsumers.containsKey(consumer)) { + continue; + } + if (consumer.getAvailablePermits() > 0) { + return true; + } + } + return false; + } + @Override public SubType getType() { return SubType.Key_Shared; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index f00c95f7e6816..c5eeab63ccd95 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -151,7 +151,7 @@ public PersistentSubscription(PersistentTopic topic, String subscriptionName, Ma this.topicName = topic.getName(); this.subName = subscriptionName; this.fullName = MoreObjects.toStringHelper(this).add("topic", topicName).add("name", subName).toString(); - this.expiryMonitor = new PersistentMessageExpiryMonitor(topic, subscriptionName, cursor, this); + this.expiryMonitor = new PersistentMessageExpiryMonitor(topicName, subscriptionName, cursor, this); this.setReplicated(replicated); this.subscriptionProperties = MapUtils.isEmpty(subscriptionProperties) ? Collections.emptyMap() : Collections.unmodifiableMap(subscriptionProperties); @@ -306,7 +306,6 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (dispatcher != null && dispatcher.getConsumers().isEmpty()) { deactivateCursor(); - topic.getManagedLedger().removeWaitingCursor(cursor); if (!cursor.isDurable()) { // If cursor is not durable, we need to clean up the subscription as well @@ -335,11 +334,14 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (!isResetCursor) { try { topic.getManagedLedger().deleteCursor(cursor.getName()); + topic.getManagedLedger().removeWaitingCursor(cursor); } catch (InterruptedException | ManagedLedgerException e) { log.warn("[{}] [{}] Failed to remove non durable cursor", topic.getName(), subName, e); } } }); + } else { + topic.getManagedLedger().removeWaitingCursor(cursor); } } @@ -878,6 +880,12 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { public void readEntryComplete(Entry entry, Object ctx) { future.complete(entry); } + + @Override + public String toString() { + return String.format("Subscription [%s-%s] async replay entries", PersistentSubscription.this.topicName, + PersistentSubscription.this.subName); + } }, null); return future; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 5bceed099c1f0..d7df6c1e5f76e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -196,7 +196,11 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal private volatile List shadowTopics; private final TopicName shadowSourceTopic; - static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; + public static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; + + public static boolean isDedupCursorName(String name) { + return DEDUPLICATION_CURSOR_NAME.equals(name); + } private static final String TOPIC_EPOCH_PROPERTY_NAME = "pulsar.topic.epoch"; private static final double MESSAGE_EXPIRY_THRESHOLD = 1.5; @@ -2938,14 +2942,14 @@ public CompletableFuture checkBacklogQuotaExceeded(String producerName, Ba if ((retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold || retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception)) { if (backlogQuotaType == BacklogQuotaType.destination_storage && isSizeBacklogExceeded()) { - log.info("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } if (backlogQuotaType == BacklogQuotaType.message_age) { return checkTimeBacklogExceeded().thenCompose(isExceeded -> { if (isExceeded) { - log.info("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } else { @@ -3119,7 +3123,7 @@ public boolean isOldestMessageExpired(ManagedCursor cursor, int messageTTLInSeco // if AutoSkipNonRecoverableData is set to true, just return true here. return true; } else { - log.warn("[{}] Error while getting the oldest message", topic, e); + log.warn("[{}] [{}] Error while getting the oldest message", topic, cursor.toString(), e); } } finally { if (entry != null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java deleted file mode 100644 index 08730189322ee..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * 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 org.apache.pulsar.broker.stats; - -import java.util.concurrent.atomic.AtomicReferenceArray; -import java.util.function.Function; - -public final class TimeWindow { - private final int interval; - private final int sampleCount; - private final AtomicReferenceArray> array; - - public TimeWindow(int sampleCount, int interval) { - this.sampleCount = sampleCount; - this.interval = interval; - this.array = new AtomicReferenceArray<>(sampleCount); - } - - /** - * return current time window data. - * - * @param function generate data. - * @return - */ - public synchronized WindowWrap current(Function function) { - long millis = System.currentTimeMillis(); - - if (millis < 0) { - return null; - } - int idx = calculateTimeIdx(millis); - long windowStart = calculateWindowStart(millis); - while (true) { - WindowWrap old = array.get(idx); - if (old == null) { - WindowWrap window = new WindowWrap<>(interval, windowStart, null); - if (array.compareAndSet(idx, null, window)) { - T value = null == function ? null : function.apply(null); - window.value(value); - return window; - } else { - Thread.yield(); - } - } else if (windowStart == old.start()) { - return old; - } else if (windowStart > old.start()) { - T value = null == function ? null : function.apply(old.value()); - old.value(value); - old.resetWindowStart(windowStart); - return old; - } else { - //it should never goes here - throw new IllegalStateException(); - } - } - } - - private int calculateTimeIdx(long timeMillis) { - long timeId = timeMillis / this.interval; - return (int) (timeId % sampleCount); - } - - private long calculateWindowStart(long timeMillis) { - return timeMillis - timeMillis % this.interval; - } - - public int sampleCount() { - return sampleCount; - } - - public int interval() { - return interval; - } - - public long currentWindowStart(long millis) { - return this.calculateWindowStart(millis); - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java deleted file mode 100644 index 12869b82921e5..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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 org.apache.pulsar.broker.stats; - -public final class WindowWrap { - private final long interval; - private long start; - private T value; - - public WindowWrap(long interval, long windowStart, T value) { - this.interval = interval; - this.start = windowStart; - this.value = value; - } - - public long interval() { - return this.interval; - } - - public long start() { - return this.start; - } - - public T value() { - return value; - } - - public void value(T value) { - this.value = value; - } - - public WindowWrap resetWindowStart(long startTime) { - this.start = startTime; - return this; - } - - public boolean isTimeInWindow(long timeMillis) { - return start <= timeMillis && timeMillis < start + interval; - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java new file mode 100644 index 0000000000000..b80e5747d8a5a --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java @@ -0,0 +1,68 @@ +/* + * 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 org.apache.pulsar.broker.stats.prometheus; + +import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Gauge; +import io.prometheus.client.hotspot.DefaultExports; +import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.common.util.DirectMemoryUtils; + +public class MetricsExports { + private static boolean initialized = false; + + private MetricsExports() { + } + + public static synchronized void initialize() { + if (!initialized) { + DefaultExports.initialize(); + register(CollectorRegistry.defaultRegistry); + initialized = true; + } + } + + public static void register(CollectorRegistry registry) { + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }).register(registry); + + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }).register(registry); + + // metric to export pulsar version info + Gauge.build("pulsar_version_info", "-") + .labelNames("version", "commit").create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return 1.0; + } + }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) + .register(registry); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index ca05fa40a2530..5a5a61404b87f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -41,7 +41,10 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray SimpleTextOutputStream stream = initGaugeType(metricName); stream.write(metricName).write('{'); for (int i = 0; i < labelsAndValuesArray.length; i += 2) { - String labelValue = PrometheusMetricsGeneratorUtils.writeEscapedLabelValue(labelsAndValuesArray[i + 1]); + String labelValue = labelsAndValuesArray[i + 1]; + if (labelValue != null && labelValue.indexOf('"') > -1) { + labelValue = labelValue.replace("\"", "\\\""); + } stream.write(labelsAndValuesArray[i]).write("=\"").write(labelValue).write('\"'); if (i + 2 != labelsAndValuesArray.length) { stream.write(','); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 293ac30ecd123..da77fe9b3341d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -20,41 +20,40 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.generateSystemMetrics; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.getTypeStr; -import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; import io.netty.buffer.ByteBuf; -import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import io.prometheus.client.Collector; -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.Gauge; -import io.prometheus.client.Gauge.Child; -import io.prometheus.client.hotspot.DefaultExports; +import java.io.BufferedOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.io.StringWriter; +import java.io.OutputStreamWriter; import java.io.Writer; -import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.StandardCharsets; +import java.time.Clock; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.TimeWindow; -import org.apache.pulsar.broker.stats.WindowWrap; import org.apache.pulsar.broker.stats.metrics.ManagedCursorMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerCacheMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.DirectMemoryUtils; import org.apache.pulsar.common.util.SimpleTextOutputStream; -import org.eclipse.jetty.server.HttpOutput; /** * Generate metrics aggregated at the namespace level and optionally at a topic level and formats them out @@ -63,123 +62,80 @@ * href="https://prometheus.io/docs/instrumenting/exposition_formats/">Exposition Formats */ @Slf4j -public class PrometheusMetricsGenerator { - private static volatile TimeWindow timeWindow; - private static final int MAX_COMPONENTS = 64; - - static { - DefaultExports.initialize(); - - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); - - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); - - // metric to export pulsar version info - Gauge.build("pulsar_version_info", "-") - .labelNames("version", "commit").create() - .setChild(new Child() { - @Override - public double get() { - return 1.0; - } - }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) - .register(CollectorRegistry.defaultRegistry); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, false, out, null); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, out, null); - } - - public static synchronized void generate(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean includeProducerMetrics, - boolean splitTopicAndPartitionIndexLabel, OutputStream out, - List metricsProviders) throws IOException { - ByteBuf buffer; - boolean exposeBufferMetrics = pulsar.getConfiguration().isMetricsBufferResponse(); +public class PrometheusMetricsGenerator implements AutoCloseable { + private static final int DEFAULT_INITIAL_BUFFER_SIZE = 1024 * 1024; // 1MB + private static final int MINIMUM_FOR_MAX_COMPONENTS = 64; + + private volatile MetricsBuffer metricsBuffer; + private static AtomicReferenceFieldUpdater metricsBufferFieldUpdater = + AtomicReferenceFieldUpdater.newUpdater(PrometheusMetricsGenerator.class, MetricsBuffer.class, + "metricsBuffer"); + private volatile boolean closed; + + public static class MetricsBuffer { + private final CompletableFuture bufferFuture; + private final long createTimeslot; + private final AtomicInteger refCnt = new AtomicInteger(2); + + MetricsBuffer(long timeslot) { + bufferFuture = new CompletableFuture<>(); + createTimeslot = timeslot; + } - if (!exposeBufferMetrics) { - buffer = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - } else { - if (null == timeWindow) { - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - timeWindow = new TimeWindow<>(1, (int) TimeUnit.SECONDS.toMillis(period)); - } - WindowWrap window = timeWindow.current(oldBuf -> { - // release expired buffer, in case of memory leak - if (oldBuf != null && oldBuf.refCnt() > 0) { - oldBuf.release(); - log.debug("Cached metrics buffer released"); - } + public CompletableFuture getBufferFuture() { + return bufferFuture; + } - try { - ByteBuf buf = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - log.debug("Generated metrics buffer size {}", buf.readableBytes()); - return buf; - } catch (IOException e) { - log.error("Generate metrics failed", e); - //return empty buffer if exception happens - return PulsarByteBufAllocator.DEFAULT.heapBuffer(0); - } - }); + long getCreateTimeslot() { + return createTimeslot; + } - if (null == window || null == window.value()) { - return; - } - buffer = window.value(); - log.debug("Current window start {}, current cached buf size {}", window.start(), buffer.readableBytes()); + /** + * Retain the buffer. This is allowed, only when the buffer is not already released. + * + * @return true if the buffer is retained successfully, false otherwise. + */ + boolean retain() { + return refCnt.updateAndGet(x -> x > 0 ? x + 1 : x) > 0; } - try { - if (out instanceof HttpOutput) { - HttpOutput output = (HttpOutput) out; - //no mem_copy and memory allocations here - ByteBuffer[] buffers = buffer.nioBuffers(); - for (ByteBuffer buffer0 : buffers) { - output.write(buffer0); - } - } else { - //read data from buffer and write it to output stream, with no more heap buffer(byte[]) allocation. - //not modify buffer readIndex/writeIndex here. - int readIndex = buffer.readerIndex(); - int readableBytes = buffer.readableBytes(); - for (int i = 0; i < readableBytes; i++) { - out.write(buffer.getByte(readIndex + i)); - } - } - } finally { - if (!exposeBufferMetrics && buffer.refCnt() > 0) { - buffer.release(); - log.debug("Metrics buffer released."); + /** + * Release the buffer. + */ + public void release() { + int newValue = refCnt.decrementAndGet(); + if (newValue == 0) { + bufferFuture.whenComplete((byteBuf, throwable) -> { + if (byteBuf != null) { + byteBuf.release(); + } + }); } } } - private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - List metricsProviders) throws IOException { - //Use unpooled buffers here to avoid direct buffer usage increasing. - //when write out 200MB data, MAX_COMPONENTS = 64 needn't mem_copy. see: CompositeByteBuf#consolidateIfNeeded() - ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); + private final PulsarService pulsar; + private final boolean includeTopicMetrics; + private final boolean includeConsumerMetrics; + private final boolean includeProducerMetrics; + private final boolean splitTopicAndPartitionIndexLabel; + private final Clock clock; + + private volatile int initialBufferSize = DEFAULT_INITIAL_BUFFER_SIZE; + + public PrometheusMetricsGenerator(PulsarService pulsar, boolean includeTopicMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, + boolean splitTopicAndPartitionIndexLabel, Clock clock) { + this.pulsar = pulsar; + this.includeTopicMetrics = includeTopicMetrics; + this.includeConsumerMetrics = includeConsumerMetrics; + this.includeProducerMetrics = includeProducerMetrics; + this.splitTopicAndPartitionIndexLabel = splitTopicAndPartitionIndexLabel; + this.clock = clock; + } + + private ByteBuf generate0(List metricsProviders) { + ByteBuf buf = allocateMultipartCompositeDirectBuffer(); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names PrometheusMetricStreams metricStreams = new PrometheusMetricStreams(); @@ -221,10 +177,34 @@ private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetri //if exception happens, release buffer if (exceptionHappens) { buf.release(); + } else { + // for the next time, the initial buffer size will be suggested by the last buffer size + initialBufferSize = Math.max(DEFAULT_INITIAL_BUFFER_SIZE, buf.readableBytes()); } } } + private ByteBuf allocateMultipartCompositeDirectBuffer() { + // use composite buffer with pre-allocated buffers to ensure that the pooled allocator can be used + // for allocating the buffers + ByteBufAllocator byteBufAllocator = PulsarByteBufAllocator.DEFAULT; + int chunkSize; + if (byteBufAllocator instanceof PooledByteBufAllocator) { + PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; + chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); + } else { + chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; + } + CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( + Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); + int totalLen = 0; + while (totalLen < initialBufferSize) { + totalLen += chunkSize; + buf.addComponent(false, byteBufAllocator.directBuffer(chunkSize)); + } + return buf; + } + private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { String clusterName = pulsar.getConfiguration().getClusterName(); // generate managedLedgerCache metrics @@ -270,12 +250,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = key.substring(0, nameIndex); value = key.substring(nameIndex + 1); if (!names.contains(name)) { - stream.write("# TYPE ").write(name.replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write("\n"); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, name); + stream.write(' ').write(getTypeStr(metricType)).write("\n"); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } catch (Exception e) { continue; } @@ -284,12 +265,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = entry.getKey(); if (!names.contains(name)) { - stream.write("# TYPE ").write(entry.getKey().replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write('\n'); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, entry.getKey()); + stream.write(' ').write(getTypeStr(metricType)).write('\n'); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } //to avoid quantile label duplicated @@ -312,18 +294,98 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, } } + private static SimpleTextOutputStream writeNameReplacingBrkPrefix(SimpleTextOutputStream stream, String name) { + if (name.startsWith("brk_")) { + return stream.write("pulsar_").write(CharBuffer.wrap(name).position("brk_".length())); + } else { + return stream.write(name); + } + } + private static void generateManagedLedgerBookieClientMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { StatsProvider statsProvider = pulsar.getManagedLedgerClientFactory().getStatsProvider(); if (statsProvider instanceof NullStatsProvider) { return; } - try { - Writer writer = new StringWriter(); + try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { + @Override + public void write(int b) throws IOException { + stream.writeByte(b); + } + + @Override + public void write(byte b[], int off, int len) throws IOException { + stream.write(b, off, len); + } + }), StandardCharsets.UTF_8)) { statsProvider.writeAllMetrics(writer); - stream.write(writer.toString()); } catch (IOException e) { - // nop + log.error("Failed to write managed ledger bookie client metrics", e); + } + } + + public MetricsBuffer renderToBuffer(Executor executor, List metricsProviders) { + boolean cacheMetricsResponse = pulsar.getConfiguration().isMetricsBufferResponse(); + while (!closed && !Thread.currentThread().isInterrupted()) { + long currentTimeSlot = cacheMetricsResponse ? calculateCurrentTimeSlot() : 0; + MetricsBuffer currentMetricsBuffer = metricsBuffer; + if (currentMetricsBuffer == null || currentMetricsBuffer.getBufferFuture().isCompletedExceptionally() + || (currentMetricsBuffer.getBufferFuture().isDone() + && (currentMetricsBuffer.getCreateTimeslot() != 0 + && currentTimeSlot > currentMetricsBuffer.getCreateTimeslot()))) { + MetricsBuffer newMetricsBuffer = new MetricsBuffer(currentTimeSlot); + if (metricsBufferFieldUpdater.compareAndSet(this, currentMetricsBuffer, newMetricsBuffer)) { + if (currentMetricsBuffer != null) { + currentMetricsBuffer.release(); + } + CompletableFuture bufferFuture = newMetricsBuffer.getBufferFuture(); + executor.execute(() -> { + try { + bufferFuture.complete(generate0(metricsProviders)); + } catch (Exception e) { + bufferFuture.completeExceptionally(e); + } finally { + if (currentTimeSlot == 0) { + // if the buffer is not cached, release it after the future is completed + metricsBufferFieldUpdater.compareAndSet(this, newMetricsBuffer, null); + newMetricsBuffer.release(); + } + } + }); + // no need to retain before returning since the new buffer starts with refCnt 2 + return newMetricsBuffer; + } else { + currentMetricsBuffer = metricsBuffer; + } + } + // retain the buffer before returning + // if the buffer is already released, retaining won't succeed, retry in that case + if (currentMetricsBuffer != null && currentMetricsBuffer.retain()) { + return currentMetricsBuffer; + } + } + return null; + } + + /** + * Calculate the current time slot based on the current time. + * This is to ensure that cached metrics are refreshed consistently at a fixed interval regardless of the request + * time. + */ + private long calculateCurrentTimeSlot() { + long cacheTimeoutMillis = + TimeUnit.SECONDS.toMillis(Math.max(1, pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds())); + long now = clock.millis(); + return now / cacheTimeoutMillis; + } + + @Override + public void close() { + closed = true; + MetricsBuffer buffer = metricsBufferFieldUpdater.getAndSet(this, null); + if (buffer != null) { + buffer.release(); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index 42bd2652883b6..7fcc74e965c24 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -18,34 +18,142 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import java.io.EOFException; import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletOutputStream; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; +import org.eclipse.jetty.server.HttpOutput; +@Slf4j public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { - private static final long serialVersionUID = 1L; + private static final int EXECUTOR_MAX_THREADS = 4; - private final PulsarService pulsar; - private final boolean shouldExportTopicMetrics; - private final boolean shouldExportConsumerMetrics; - private final boolean shouldExportProducerMetrics; - private final boolean splitTopicAndPartitionLabel; + private final PrometheusMetricsGenerator prometheusMetricsGenerator; public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean shouldExportProducerMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean splitTopicAndPartitionLabel) { - super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName()); - this.pulsar = pulsar; - this.shouldExportTopicMetrics = includeTopicMetrics; - this.shouldExportConsumerMetrics = includeConsumerMetrics; - this.shouldExportProducerMetrics = shouldExportProducerMetrics; - this.splitTopicAndPartitionLabel = splitTopicAndPartitionLabel; + super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName(), + EXECUTOR_MAX_THREADS); + MetricsExports.initialize(); + prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionLabel, Clock.systemUTC()); } + @Override - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGenerator.generate(pulsar, shouldExportTopicMetrics, shouldExportConsumerMetrics, - shouldExportProducerMetrics, splitTopicAndPartitionLabel, outputStream, metricsProviders); + public void destroy() { + super.destroy(); + prometheusMetricsGenerator.close(); + } + + protected void doGet(HttpServletRequest request, HttpServletResponse response) { + AsyncContext context = request.startAsync(); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean skipWritingResponse = new AtomicBoolean(false); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent event) throws IOException { + } + + @Override + public void onTimeout(AsyncEvent event) throws IOException { + log.warn("Prometheus metrics request timed out"); + skipWritingResponse.set(true); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent event) throws IOException { + skipWritingResponse.set(true); + } + + @Override + public void onStartAsync(AsyncEvent event) throws IOException { + } + }); + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + prometheusMetricsGenerator.renderToBuffer(executor, metricsProviders); + if (metricsBuffer == null) { + log.info("Service is closing, skip writing metrics."); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + context.complete(); + return; + } + metricsBuffer.getBufferFuture().whenComplete((buffer, ex) -> executor.execute(() -> { + try { + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos( + metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted() && !skipWritingResponse.get()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + return; + } + if (skipWritingResponse.get()) { + log.warn("Response has timed or failed, skip writing metrics."); + return; + } + if (response.isCommitted()) { + log.warn("Response is already committed, cannot write metrics"); + return; + } + if (ex != null) { + log.error("Failed to generate metrics", ex); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + return; + } + if (buffer == null) { + log.error("Failed to generate metrics, buffer is null"); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } else { + response.setStatus(HTTP_STATUS_OK_200); + response.setContentType("text/plain;charset=utf-8"); + ServletOutputStream outputStream = response.getOutputStream(); + if (outputStream instanceof HttpOutput) { + HttpOutput output = (HttpOutput) outputStream; + for (ByteBuffer nioBuffer : buffer.nioBuffers()) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(outputStream, length); + } + } + } + } catch (EOFException e) { + log.error("Failed to write metrics to response due to EOFException"); + } catch (IOException e) { + log.error("Failed to write metrics to response", e); + } finally { + metricsBuffer.release(); + context.complete(); + } + })); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index dda03e3e59dd4..33ef05df9ebc4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -475,7 +475,9 @@ private static void writeConsumerMetric(PrometheusMetricStreams stream, String m static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel, String... extraLabelsAndValues) { - String[] labelsAndValues = new String[splitTopicAndPartitionIndexLabel ? 8 : 6]; + int baseLabelCount = splitTopicAndPartitionIndexLabel ? 8 : 6; + String[] labelsAndValues = + new String[baseLabelCount + (extraLabelsAndValues != null ? extraLabelsAndValues.length : 0)]; labelsAndValues[0] = "cluster"; labelsAndValues[1] = cluster; labelsAndValues[2] = "namespace"; @@ -495,7 +497,11 @@ static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, } else { labelsAndValues[5] = topic; } - String[] labels = ArrayUtils.addAll(labelsAndValues, extraLabelsAndValues); - stream.writeSample(metricName, value, labels); + if (extraLabelsAndValues != null) { + for (int i = 0; i < extraLabelsAndValues.length; i++) { + labelsAndValues[baseLabelCount + i] = extraLabelsAndValues[i]; + } + } + stream.writeSample(metricName, value, labelsAndValues); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java index be1271a155cd3..385500dfbe9e7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java @@ -327,6 +327,12 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { hasInvalidIndex.set(true); } } + + @Override + public String toString() { + return String.format("Transaction buffer [%s] recover from snapshot", + SnapshotSegmentAbortedTxnProcessorImpl.this.topic.getName()); + } }, null); }); openManagedLedgerAndHandleSegmentsFuture.complete(null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 7dbe0385fd7e9..5ed271c6fd414 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.compareToWithAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetOverlap; import com.google.common.annotations.VisibleForTesting; +import io.netty.util.Timer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -35,9 +36,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -45,6 +48,7 @@ import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; import org.apache.pulsar.broker.service.Consumer; @@ -53,7 +57,9 @@ import org.apache.pulsar.broker.transaction.pendingack.PendingAckHandleStats; import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; @@ -134,6 +140,12 @@ public class PendingAckHandleImpl extends PendingAckHandleState implements Pendi public final RecoverTimeRecord recoverTime = new RecoverTimeRecord(); + private final long pendingAckInitFailureBackoffInitialTimeInMs = 100; + + public final Backoff backoff = new Backoff(pendingAckInitFailureBackoffInitialTimeInMs, TimeUnit.MILLISECONDS, + 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + + private final Timer transactionOpTimer; public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { super(State.None); @@ -153,7 +165,11 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { this.pendingAckStoreProvider = this.persistentSubscription.getTopic() .getBrokerService().getPulsar().getTransactionPendingAckStoreProvider(); + transactionOpTimer = persistentSubscription.getTopic().getBrokerService().getPulsar().getTransactionTimer(); + init(); + } + private void init() { pendingAckStoreProvider.checkInitializedBefore(persistentSubscription) .thenAcceptAsync(init -> { if (init) { @@ -164,9 +180,9 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { }, internalPinnedExecutor) .exceptionallyAsync(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - changeToErrorState(); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(t); - this.pendingAckStoreFuture.completeExceptionally(t); return null; }, internalPinnedExecutor); } @@ -180,9 +196,8 @@ private void initPendingAckStore() { recoverTime.setRecoverStartTime(System.currentTimeMillis()); pendingAckStore.replayAsync(this, internalPinnedExecutor); }).exceptionallyAsync(e -> { - handleCacheRequest(); - changeToErrorState(); - log.error("PendingAckHandleImpl init fail! TopicName : {}, SubName: {}", topicName, subName, e); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(e.getCause()); return null; }, internalPinnedExecutor); @@ -945,12 +960,39 @@ public void completeHandleFuture() { } public void exceptionHandleFuture(Throwable t) { - final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally(t); + if (isRetryableException(t)) { + this.state = State.None; + long retryTime = backoff.next(); + log.warn("[{}][{}] Failed to init transaction pending ack. It will be retried in {} Ms", + persistentSubscription.getTopic().getName(), subName, retryTime, t); + transactionOpTimer.newTimeout((timeout) -> init(), retryTime, TimeUnit.MILLISECONDS); + return; + } + log.error("[{}] [{}] PendingAckHandleImpl init fail!", topicName, subName, t); + handleCacheRequest(); + changeToErrorState(); + // ToDo: Add a new serverError `TransactionComponentLoadFailedException` + // and before that a `Unknown` will be returned first. + this.pendingAckStoreFuture = FutureUtil.failedFuture(new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); + final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally( + new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); if (completedNow) { recoverTime.setRecoverEndTime(System.currentTimeMillis()); } } + private static boolean isRetryableException(Throwable ex) { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + return (realCause instanceof ManagedLedgerException + && !(realCause instanceof ManagedLedgerException.ManagedLedgerFencedException) + && !(realCause instanceof ManagedLedgerException.NonRecoverableLedgerException)) + || realCause instanceof PulsarClientException.BrokerPersistenceException + || realCause instanceof PulsarClientException.LookupException + || realCause instanceof PulsarClientException.ConnectException; + } + @Override public TransactionInPendingAckStats getTransactionInPendingAckStats(TxnID txnID) { TransactionInPendingAckStats transactionInPendingAckStats = new TransactionInPendingAckStats(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index eada0436f4d61..7395144167af9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -259,15 +259,16 @@ public void addFilters(ServletContextHandler context, boolean requiresAuthentica public void addServlet(String path, ServletHolder servletHolder, boolean requiresAuthentication, Map attributeMap) { - ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); + ServletContextHandler servletContextHandler = new ServletContextHandler(ServletContextHandler.SESSIONS); // Notice: each context path should be unique, but there's nothing here to verify that - context.setContextPath(path); - context.addServlet(servletHolder, MATCH_ALL); + servletContextHandler.setContextPath(path); + servletContextHandler.addServlet(servletHolder, MATCH_ALL); if (attributeMap != null) { - attributeMap.forEach(context::setAttribute); + attributeMap.forEach(servletContextHandler::setAttribute); } - filterInitializer.addFilters(context, requiresAuthentication); - handlers.add(context); + filterInitializer.addFilters(servletContextHandler, requiresAuthentication); + + handlers.add(servletContextHandler); } public void addStaticResources(String basePath, String resourcePath) { @@ -291,8 +292,10 @@ public void start() throws PulsarServerException { ContextHandlerCollection contexts = new ContextHandlerCollection(); contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); + Handler handlerForContexts = GzipHandlerUtil.wrapWithGzipHandler(contexts, + pulsar.getConfig().getHttpServerGzipCompressionExcludedPaths()); HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[] { contexts, new DefaultHandler(), requestLogHandler }); + handlerCollection.setHandlers(new Handler[] {handlerForContexts, new DefaultHandler(), requestLogHandler}); // Metrics handler StatisticsHandler stats = new StatisticsHandler(); @@ -303,7 +306,6 @@ public void start() throws PulsarServerException { } catch (IllegalArgumentException e) { // Already registered. Eg: in unit tests } - handlers.add(stats); server.setHandler(stats); server.start(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java index 92a2c89f9bc9c..b7805c36b3bf9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java @@ -32,8 +32,14 @@ public interface RawReader { */ static CompletableFuture create(PulsarClient client, String topic, String subscription) { + return create(client, topic, subscription, true); + } + + static CompletableFuture create(PulsarClient client, String topic, String subscription, + boolean createTopicIfDoesNotExist) { CompletableFuture> future = new CompletableFuture<>(); - RawReader r = new RawReaderImpl((PulsarClientImpl) client, topic, subscription, future); + RawReader r = + new RawReaderImpl((PulsarClientImpl) client, topic, subscription, future, createTopicIfDoesNotExist); return future.thenApply(__ -> r); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java index f65232413991f..3d7ad9f58657d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java @@ -51,7 +51,8 @@ public class RawReaderImpl implements RawReader { private RawConsumerImpl consumer; public RawReaderImpl(PulsarClientImpl client, String topic, String subscription, - CompletableFuture> consumerFuture) { + CompletableFuture> consumerFuture, + boolean createTopicIfDoesNotExist) { consumerConfiguration = new ConsumerConfigurationData<>(); consumerConfiguration.getTopicNames().add(topic); consumerConfiguration.setSubscriptionName(subscription); @@ -61,8 +62,7 @@ public RawReaderImpl(PulsarClientImpl client, String topic, String subscription, consumerConfiguration.setSubscriptionInitialPosition(SubscriptionInitialPosition.Earliest); consumerConfiguration.setAckReceiptEnabled(true); - consumer = new RawConsumerImpl(client, consumerConfiguration, - consumerFuture); + consumer = new RawConsumerImpl(client, consumerConfiguration, consumerFuture, createTopicIfDoesNotExist); } @Override @@ -111,7 +111,7 @@ static class RawConsumerImpl extends ConsumerImpl { final Queue> pendingRawReceives; RawConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, - CompletableFuture> consumerFuture) { + CompletableFuture> consumerFuture, boolean createTopicIfDoesNotExist) { super(client, conf.getSingleTopic(), conf, @@ -123,7 +123,7 @@ static class RawConsumerImpl extends ConsumerImpl { MessageId.earliest, 0 /* startMessageRollbackDurationInSec */, Schema.BYTES, null, - false + createTopicIfDoesNotExist ); incomingRawMessages = new GrowableArrayBlockingQueue<>(); pendingRawReceives = new ConcurrentLinkedQueue<>(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java index e93a642c76e4d..983443432ff49 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/Compactor.java @@ -56,7 +56,7 @@ public Compactor(ServiceConfiguration conf, } public CompletableFuture compact(String topic) { - return RawReader.create(pulsar, topic, COMPACTION_SUBSCRIPTION).thenComposeAsync( + return RawReader.create(pulsar, topic, COMPACTION_SUBSCRIPTION, false).thenComposeAsync( this::compactAndCloseReader, scheduler); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index 647c34a94ad81..a03662bf1b803 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -312,7 +312,6 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map promise.complete(null); } }); - return; } } catch (InterruptedException ie) { Thread.currentThread().interrupt(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java new file mode 100644 index 0000000000000..fcc3b6aa88fb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -0,0 +1,84 @@ +/* + * 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 org.apache.pulsar; + +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; +import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; +import org.eclipse.jetty.server.HttpOutput; + +public class PrometheusMetricsTestUtil { + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, false, Clock.systemUTC()), out, null); + } + + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionIndexLabel, Clock.systemUTC()), out, null); + } + + public static void generate(PrometheusMetricsGenerator metricsGenerator, OutputStream out, + List metricsProviders) throws IOException { + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + metricsGenerator.renderToBuffer(MoreExecutors.directExecutor(), metricsProviders); + try { + ByteBuf buffer = null; + try { + buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } + if (buffer == null) { + return; + } + if (out instanceof HttpOutput) { + HttpOutput output = (HttpOutput) out; + ByteBuffer[] nioBuffers = buffer.nioBuffers(); + for (ByteBuffer nioBuffer : nioBuffers) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(out, length); + } + } + } finally { + metricsBuffer.release(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index c7b066faea9da..98a33274c94e0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -35,6 +35,7 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.assertThrows; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashMap; @@ -78,11 +79,13 @@ import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.interceptor.ProducerInterceptor; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -1780,6 +1783,34 @@ public void testNamespaceResources() throws Exception { assertTrue(namespaces.contains(ns1V1)); } + @Test + public void testForceDeleteSubscription() throws Exception { + try { + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(false); + String topicName = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testForceDeleteSubscription"; + String subName = "sub1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subName, MessageId.latest); + + @Cleanup + Consumer c0 = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + admin.topics().deleteSubscription(topicName, subName, true); + } finally { + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); + } + } + @Test public void testUpdatePropertiesOnNonDurableSub() throws Exception { String topic = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testUpdatePropertiesOnNonDurableSub"; @@ -1811,4 +1842,10 @@ public void testUpdatePropertiesOnNonDurableSub() throws Exception { assertEquals(cursor.getCursorProperties().size(), 1); assertEquals(cursor.getCursorProperties().get("foo"), "bar"); } + + @Test + public void testCreateMissingPartitions() throws Exception { + String topicName = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testCreateMissingPartitions"; + assertThrows(PulsarAdminException.NotFoundException.class, () -> admin.topics().createMissedPartitions(topicName)); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 03ea937c44edc..169ff89fe3c0d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -97,7 +97,6 @@ import org.apache.pulsar.broker.lookup.LookupResult; import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceBundleOwnershipListener; -import org.apache.pulsar.broker.namespace.NamespaceBundleSplitListener; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -368,23 +367,6 @@ public void testSplitBundleAdminAPI() throws Exception { String firstBundle = bundleRanges.get(0) + "_" + bundleRanges.get(1); - AtomicInteger splitCount = new AtomicInteger(0); - NamespaceBundleSplitListener namespaceBundleSplitListener = new NamespaceBundleSplitListener() { - @Override - public void onSplit(NamespaceBundle bundle) { - splitCount.incrementAndGet(); - } - - @Override - public boolean test(NamespaceBundle namespaceBundle) { - return namespaceBundle - .toString() - .equals(String.format(namespace + "/0x%08x_0x%08x", bundleRanges.get(0), bundleRanges.get(1))); - } - }; - pulsar1.getNamespaceService().addNamespaceBundleSplitListener(namespaceBundleSplitListener); - pulsar2.getNamespaceService().addNamespaceBundleSplitListener(namespaceBundleSplitListener); - long mid = bundleRanges.get(0) + (bundleRanges.get(1) - bundleRanges.get(0)) / 2; admin.namespaces().splitNamespaceBundle(namespace, firstBundle, true, null); @@ -397,7 +379,6 @@ public boolean test(NamespaceBundle namespaceBundle) { assertTrue(bundlesData.getBoundaries().contains(lowBundle)); assertTrue(bundlesData.getBoundaries().contains(midBundle)); assertTrue(bundlesData.getBoundaries().contains(highBundle)); - assertEquals(splitCount.get(), 1); // Test split bundle with invalid bundle range. try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java index 73cfaf1b0d96b..43d37466918ce 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceCreateBundlesTest.java @@ -20,21 +20,15 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; - -import lombok.Cleanup; import java.util.UUID; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; -import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.Policies; -import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -87,31 +81,4 @@ public void testSplitBundleUpdatesLocalPoliciesWithoutOverwriting() throws Excep assertNotNull(admin.namespaces().getBookieAffinityGroup(namespaceName)); producer.close(); } - - @Test - public void testBundleSplitListener() throws Exception { - String namespaceName = "prop/" + UUID.randomUUID().toString(); - String topicName = "persistent://" + namespaceName + "/my-topic5"; - admin.namespaces().createNamespace(namespaceName); - @Cleanup - Producer producer = pulsarClient.newProducer().topic(topicName).sendTimeout(1, - TimeUnit.SECONDS).create(); - producer.send(new byte[1]); - String bundleRange = admin.lookups().getBundleRange(topicName); - AtomicBoolean isTriggered = new AtomicBoolean(false); - pulsar.getNamespaceService().addNamespaceBundleSplitListener(new NamespaceBundleSplitListener() { - @Override - public void onSplit(NamespaceBundle bundle) { - assertEquals(bundleRange, bundle.getBundleRange()); - isTriggered.set(true); - } - - @Override - public boolean test(NamespaceBundle namespaceBundle) { - return true; - } - }); - admin.namespaces().splitNamespaceBundle(namespaceName, bundleRange, false, null); - Awaitility.await().untilAsserted(() -> assertTrue(isTriggered.get())); - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java index 9bf7e3c5325d9..54c23cacc0d7b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java @@ -20,6 +20,10 @@ import com.google.common.collect.Sets; import io.prometheus.client.Summary; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Map; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.ResourceGroupMonitoringClass; @@ -45,11 +49,6 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Map; -import java.util.concurrent.TimeUnit; - // The tests implement a set of producer/consumer operations on a set of topics. // [A thread is started for each producer, and each consumer in the test.] @@ -57,6 +56,7 @@ // After sending/receiving all the messages, traffic usage statistics, and Prometheus-metrics // are verified on the RGs. @Slf4j +@Test(groups = "flaky") public class RGUsageMTAggrWaitForAllMsgsTest extends ProducerConsumerBase { @BeforeClass @Override @@ -119,9 +119,9 @@ private class ProduceMessages implements Runnable { private final int numMesgsToProduce; private final String myProduceTopic; - private int sentNumBytes = 0; - private int sentNumMsgs = 0; - private int numExceptions = 0; + private volatile int sentNumBytes = 0; + private volatile int sentNumMsgs = 0; + private volatile int numExceptions = 0; ProduceMessages(int prodId, int nMesgs, String[] topics) { producerId = prodId; @@ -202,9 +202,9 @@ private class ConsumeMessages implements Runnable { private final int recvTimeoutMilliSecs = 1000; private final int ackTimeoutMilliSecs = 1100; // has to be more than 1 second - private int recvdNumBytes = 0; - private int recvdNumMsgs = 0; - private int numExceptions = 0; + private volatile int recvdNumBytes = 0; + private volatile int recvdNumMsgs = 0; + private volatile int numExceptions = 0; private volatile boolean allMessagesReceived = false; private volatile boolean consumerIsReady = false; @@ -494,15 +494,15 @@ private void testProduceConsumeUsageOnRG(String[] topicStrings) throws Exception while (numConsumersDone < NUM_CONSUMERS) { for (int ix = 0; ix < NUM_CONSUMERS; ix++) { if (!joinedConsumers[ix]) { + consThr[ix].thread.join(); + joinedConsumers[ix] = true; + log.debug("Joined consumer={}", ix); + recvdBytes = consThr[ix].consumer.getNumBytesRecvd(); recvdMsgs = consThr[ix].consumer.getNumMessagesRecvd(); numConsumerExceptions += consThr[ix].consumer.getNumExceptions(); log.debug("Consumer={} received {} mesgs and {} bytes", ix, recvdMsgs, recvdBytes); - consThr[ix].thread.join(); - joinedConsumers[ix] = true; - log.debug("Joined consumer={}", ix); - recvdNumBytes += recvdBytes; recvdNumMsgs += recvdMsgs; numConsumersDone++; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index f3463ee121d75..3918dcbe86d66 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -127,8 +127,8 @@ void setup() throws Exception { config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); config.setManagedLedgerMinLedgerRolloverTimeMinutes(0); config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - config.setSystemTopicEnabled(false); - config.setTopicLevelPoliciesEnabled(false); + config.setSystemTopicEnabled(true); + config.setTopicLevelPoliciesEnabled(true); config.setForceDeleteNamespaceAllowed(true); pulsar = new PulsarService(config); @@ -228,7 +228,7 @@ public void testBacklogQuotaWithReader() throws Exception { // non-durable mes should still assertEquals(stats.getSubscriptions().size(), 1); long nonDurableSubscriptionBacklog = stats.getSubscriptions().values().iterator().next().getMsgBacklog(); - assertEquals(nonDurableSubscriptionBacklog, MAX_ENTRIES_PER_LEDGER, + assertEquals(nonDurableSubscriptionBacklog, 0, "non-durable subscription backlog is [" + nonDurableSubscriptionBacklog + "]"); MessageIdImpl msgId = null; @@ -254,9 +254,6 @@ public void testBacklogQuotaWithReader() throws Exception { // check there is only one ledger left assertEquals(internalStats.ledgers.size(), 1); - - // check if its the expected ledger id given MAX_ENTRIES_PER_LEDGER - assertEquals(internalStats.ledgers.get(0).ledgerId, finalMsgId.getLedgerId()); }); // check reader can still read with out error @@ -303,10 +300,10 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { TopicStats stats = getTopicStats(topic1); // overall backlogSize should be zero because we only have readers assertEquals(stats.getBacklogSize(), 0, "backlog size is [" + stats.getBacklogSize() + "]"); - // non-durable mes should still assertEquals(stats.getSubscriptions().size(), 1); long nonDurableSubscriptionBacklog = stats.getSubscriptions().values().iterator().next().getMsgBacklog(); - assertEquals(nonDurableSubscriptionBacklog, MAX_ENTRIES_PER_LEDGER, + // All the full ledgers should be deleted. + assertEquals(nonDurableSubscriptionBacklog, 0, "non-durable subscription backlog is [" + nonDurableSubscriptionBacklog + "]"); MessageIdImpl messageId = null; try { @@ -327,8 +324,8 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { // check there is only one ledger left assertEquals(internalStats.ledgers.size(), 1); - // check if its the expected ledger id given MAX_ENTRIES_PER_LEDGER - assertEquals(internalStats.ledgers.get(0).ledgerId, finalMessageId.getLedgerId()); + // check if it's the expected ledger id given MAX_ENTRIES_PER_LEDGER + assertEquals(internalStats.ledgers.get(0).ledgerId, finalMessageId.getLedgerId() + 1); }); // check reader can still read with out error @@ -1169,8 +1166,13 @@ public void testProducerException() throws Exception { assertTrue(gotException, "backlog exceeded exception did not occur"); } - @Test - public void testProducerExceptionAndThenUnblockSizeQuota() throws Exception { + @DataProvider(name = "dedupTestSet") + public static Object[][] dedupTestSet() { + return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; + } + + @Test(dataProvider = "dedupTestSet") + public void testProducerExceptionAndThenUnblockSizeQuota(boolean dedupTestSet) throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/quotahold"), new HashMap<>()); admin.namespaces().setBacklogQuota("prop/quotahold", @@ -1186,9 +1188,12 @@ public void testProducerExceptionAndThenUnblockSizeQuota() throws Exception { boolean gotException = false; Consumer consumer = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); - byte[] content = new byte[1024]; Producer producer = createProducer(client, topic1); + + admin.topicPolicies().setDeduplicationStatus(topic1, dedupTestSet); + Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); + for (int i = 0; i < 10; i++) { producer.send(content); } @@ -1207,6 +1212,7 @@ public void testProducerExceptionAndThenUnblockSizeQuota() throws Exception { } assertTrue(gotException, "backlog exceeded exception did not occur"); + assertFalse(producer.isConnected()); // now remove backlog and ensure that producer is unblocked; TopicStats stats = getTopicStats(topic1); @@ -1223,14 +1229,33 @@ public void testProducerExceptionAndThenUnblockSizeQuota() throws Exception { Exception sendException = null; gotException = false; try { - for (int i = 0; i < 5; i++) { + for (int i = 0; i < 10; i++) { producer.send(content); + Message msg = consumer.receive(); + consumer.acknowledge(msg); } } catch (Exception e) { gotException = true; sendException = e; } + Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); assertFalse(gotException, "unable to publish due to " + sendException); + + gotException = false; + long lastDisconnectedTimestamp = producer.getLastDisconnectedTimestamp(); + try { + // try to send over backlog quota and make sure it passes + producer.send(content); + producer.send(content); + } catch (PulsarClientException ce) { + assertTrue(ce instanceof PulsarClientException.ProducerBlockedQuotaExceededException + || ce instanceof PulsarClientException.TimeoutException, ce.getMessage()); + gotException = true; + sendException = ce; + } + assertFalse(gotException, "unable to publish due to " + sendException); + assertEquals(lastDisconnectedTimestamp, producer.getLastDisconnectedTimestamp()); + } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesChaosTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesChaosTest.java new file mode 100644 index 0000000000000..d49489d8a84b0 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesChaosTest.java @@ -0,0 +1,71 @@ +/* + * 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 org.apache.pulsar.broker.service; + +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.Producer; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class BkEnsemblesChaosTest extends CanReconnectZKClientPulsarServiceBaseTest { + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Test + public void testBookieInfoIsCorrectEvenIfLostNotificationDueToZKClientReconnect() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final byte[] msgValue = "test".getBytes(); + admin.topics().createNonPartitionedTopic(topicName); + // Ensure broker works. + Producer producer1 = client.newProducer().topic(topicName).create(); + producer1.send(msgValue); + producer1.close(); + admin.topics().unload(topicName); + + // Restart some bookies, which triggers the ZK node of Bookie deleted and created. + // And make the local metadata store reconnect to lose some notification of the ZK node change. + for (int i = 0; i < numberOfBookies - 1; i++){ + bkEnsemble.stopBK(i); + } + makeLocalMetadataStoreKeepReconnect(); + for (int i = 0; i < numberOfBookies - 1; i++){ + bkEnsemble.startBK(i); + } + // Sleep 100ms to lose the notifications of ZK node create. + Thread.sleep(100); + stopLocalMetadataStoreAlwaysReconnect(); + + // Ensure broker still works. + admin.topics().unload(topicName); + Producer producer2 = client.newProducer().topic(topicName).create(); + producer2.send(msgValue); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java index 40649a4164047..42b9358911a69 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java @@ -191,9 +191,9 @@ public void testSkipCorruptDataLedger() throws Exception { .build(); final String ns1 = "prop/usc/crash-broker"; - final int totalMessages = 100; + final int totalMessages = 99; final int totalDataLedgers = 5; - final int entriesPerLedger = totalMessages / totalDataLedgers; + final int entriesPerLedger = 20; try { admin.namespaces().createNamespace(ns1); @@ -273,9 +273,9 @@ public void testSkipCorruptDataLedger() throws Exception { retryStrategically((test) -> config.isAutoSkipNonRecoverableData(), 5, 100); - // (5) consumer will be able to consume 20 messages from last non-deleted ledger + // (5) consumer will be able to consume 19 messages from last non-deleted ledger consumer = client.newConsumer().topic(topic1).subscriptionName("my-subscriber-name").subscribe(); - for (int i = 0; i < entriesPerLedger; i++) { + for (int i = 0; i < entriesPerLedger - 1; i++) { msg = consumer.receive(); System.out.println(i); consumer.acknowledge(msg); @@ -296,9 +296,9 @@ public void testTruncateCorruptDataLedger() throws Exception { .statsInterval(0, TimeUnit.SECONDS) .build(); - final int totalMessages = 100; + final int totalMessages = 99; final int totalDataLedgers = 5; - final int entriesPerLedger = totalMessages / totalDataLedgers; + final int entriesPerLedger = 20; final String tenant = "prop"; try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java index 5252407892eea..3b2f3cf215ea3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java @@ -231,28 +231,43 @@ public void testBookieIsolation() throws Exception { LedgerManager ledgerManager = getLedgerManager(bookie1); // namespace: ns1 - ManagedLedgerImpl ml = (ManagedLedgerImpl) topic1.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml1 = (ManagedLedgerImpl) topic1.getManagedLedger(); + // totalLedgers = totalPublish / totalEntriesPerLedger. (totalPublish = 100, totalEntriesPerLedger = 20.) + // The last ledger is full, a new empty ledger will be created. + // The ledger is created async, so adding a wait is needed. + Awaitility.await().untilAsserted(() -> { + assertEquals(ml1.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml1.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), defaultBookies); + assertAffinityBookies(ledgerManager, ml1.getLedgersInfoAsList(), defaultBookies); // namespace: ns2 - ml = (ManagedLedgerImpl) topic2.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml2.getLedgersInfoAsList(), isolatedBookies); // namespace: ns3 - ml = (ManagedLedgerImpl) topic3.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml3 = (ManagedLedgerImpl) topic3.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml3.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml3.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); // namespace: ns4 - ml = (ManagedLedgerImpl) topic4.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml4 = (ManagedLedgerImpl) topic4.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml4.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml4.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); @@ -388,11 +403,14 @@ public void testSetRackInfoAndAffinityGroupDuringProduce() throws Exception { ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); // namespace: ns2 - assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers); - + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); List ledgers = ml2.getLedgersInfoAsList(); // validate ledgers' ensemble with affinity bookies - for (int i=1; i> ledgerMetaFuture = ledgerManager.readLedgerMetadata(ledgerId); @@ -529,28 +547,40 @@ public void testStrictBookieIsolation() throws Exception { LedgerManager ledgerManager = getLedgerManager(bookie1); // namespace: ns1 - ManagedLedgerImpl ml = (ManagedLedgerImpl) topic1.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml1 = (ManagedLedgerImpl) topic1.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml1.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml1.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), defaultBookies); + assertAffinityBookies(ledgerManager, ml1.getLedgersInfoAsList(), defaultBookies); // namespace: ns2 - ml = (ManagedLedgerImpl) topic2.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml2.getLedgersInfoAsList(), isolatedBookies); // namespace: ns3 - ml = (ManagedLedgerImpl) topic3.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml3 = (ManagedLedgerImpl) topic3.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml3.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml3.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); // namespace: ns4 - ml = (ManagedLedgerImpl) topic4.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml4 = (ManagedLedgerImpl) topic4.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml4.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml4.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); @@ -687,22 +717,32 @@ public void testBookieIsolationWithSecondaryGroup() throws Exception { LedgerManager ledgerManager = getLedgerManager(bookie1); // namespace: ns1 - ManagedLedgerImpl ml = (ManagedLedgerImpl) topic1.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml1 = (ManagedLedgerImpl) topic1.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml1.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml1.getCurrentLedgerEntries(), 0); + }); + // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), defaultBookies); + assertAffinityBookies(ledgerManager, ml1.getLedgersInfoAsList(), defaultBookies); // namespace: ns2 - ml = (ManagedLedgerImpl) topic2.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml2.getLedgersInfoAsList(), isolatedBookies); // namespace: ns3 - ml = (ManagedLedgerImpl) topic3.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml3 = (ManagedLedgerImpl) topic3.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml3.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml3.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java index 3f2e182874e1d..f1128e389ca45 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoSubscriptionCreationTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; +import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; @@ -159,15 +160,19 @@ public void testDynamicConfigurationTopicAutoSubscriptionCreation() throws PulsarAdminException, PulsarClientException { pulsar.getConfiguration().setAllowAutoTopicCreation(false); pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); - admin.brokers().updateDynamicConfiguration("allowAutoSubscriptionCreation", "false"); + String allowAutoSubscriptionCreation = "allowAutoSubscriptionCreation"; + admin.brokers().updateDynamicConfiguration(allowAutoSubscriptionCreation, "false"); String topicString = "persistent://prop/ns-abc/non-partitioned-topic" + UUID.randomUUID(); String subscriptionName = "non-partitioned-topic-sub"; admin.topics().createNonPartitionedTopic(topicString); Assert.assertThrows(PulsarClientException.class, ()-> pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe()); - admin.brokers().updateDynamicConfiguration("allowAutoSubscriptionCreation", "true"); - pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe(); - assertTrue(admin.topics().getSubscriptions(topicString).contains(subscriptionName)); + admin.brokers().updateDynamicConfiguration(allowAutoSubscriptionCreation, "true"); + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(admin.brokers().getAllDynamicConfigurations().get(allowAutoSubscriptionCreation), "true"); + pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe(); + assertTrue(admin.topics().getSubscriptions(topicString).contains(subscriptionName)); + }); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceChaosTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceChaosTest.java new file mode 100644 index 0000000000000..4187364e46f65 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceChaosTest.java @@ -0,0 +1,103 @@ +/* + * 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 org.apache.pulsar.broker.service; + +import static org.testng.Assert.assertEquals; +import java.nio.charset.StandardCharsets; +import java.util.UUID; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class BrokerServiceChaosTest extends CanReconnectZKClientPulsarServiceBaseTest { + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Test + public void testFetchPartitionedTopicMetadataWithCacheRefresh() throws Exception { + final String configMetadataStoreConnectString = + WhiteboxImpl.getInternalState(pulsar.getConfigurationMetadataStore(), "zkConnectString"); + final ZooKeeper anotherZKCli = new ZooKeeper(configMetadataStoreConnectString, 5000, null); + // Set policy of auto create topic to PARTITIONED. + final String ns = defaultTenant + "/ns_" + UUID.randomUUID().toString().replaceAll("-", ""); + final TopicName topicName1 = TopicName.get("persistent://" + ns + "/tp1"); + final TopicName topicName2 = TopicName.get("persistent://" + ns + "/tp2"); + admin.namespaces().createNamespace(ns); + AutoTopicCreationOverride autoTopicCreationOverride = + new AutoTopicCreationOverrideImpl.AutoTopicCreationOverrideImplBuilder().allowAutoTopicCreation(true) + .topicType(TopicType.PARTITIONED.toString()) + .defaultNumPartitions(3).build(); + admin.namespaces().setAutoTopicCreationAsync(ns, autoTopicCreationOverride); + // Make the cache of namespace policy is valid. + admin.namespaces().getAutoSubscriptionCreation(ns); + // Trigger the zk node "/admin/partitioned-topics/{namespace}/persistent" created. + admin.topics().createPartitionedTopic(topicName1.toString(), 2); + admin.topics().deletePartitionedTopic(topicName1.toString()); + + // Since there is no partitioned metadata created, the partitions count of metadata will be 0. + PartitionedTopicMetadata partitionedTopicMetadata1 = + pulsar.getBrokerService().fetchPartitionedTopicMetadataAsync(topicName2).get(); + assertEquals(partitionedTopicMetadata1.partitions, 0); + + // Create the partitioned metadata by another zk client. + // Make a error to make the cache could not update. + makeLocalMetadataStoreKeepReconnect(); + anotherZKCli.create("/admin/partitioned-topics/" + ns + "/persistent/" + topicName2.getLocalName(), + "{\"partitions\":3}".getBytes(StandardCharsets.UTF_8), + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + stopLocalMetadataStoreAlwaysReconnect(); + + // Get the partitioned metadata from cache, there is 90% chance that partitions count of metadata is 0. + PartitionedTopicMetadata partitionedTopicMetadata2 = + pulsar.getBrokerService().fetchPartitionedTopicMetadataAsync(topicName2).get(); + // Note: If you want to reproduce the issue, you can perform validation on the next line. + // assertEquals(partitionedTopicMetadata2.partitions, 0); + + // Verify the new method will return a correct result. + PartitionedTopicMetadata partitionedTopicMetadata3 = + pulsar.getBrokerService().fetchPartitionedTopicMetadataAsync(topicName2, true).get(); + assertEquals(partitionedTopicMetadata3.partitions, 3); + + // cleanup. + admin.topics().deletePartitionedTopic(topicName2.toString()); + anotherZKCli.close(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java new file mode 100644 index 0000000000000..bc6df685ffcd7 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java @@ -0,0 +1,215 @@ +/* + * 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 org.apache.pulsar.broker.service; + +import com.google.common.collect.Sets; +import io.netty.channel.Channel; +import java.net.URL; +import java.nio.channels.SelectionKey; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.pulsar.tests.TestRetrySupport; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.apache.zookeeper.ClientCnxn; +import org.apache.zookeeper.ZooKeeper; +import org.awaitility.reflect.WhiteboxImpl; + +@Slf4j +public abstract class CanReconnectZKClientPulsarServiceBaseTest extends TestRetrySupport { + + protected final String defaultTenant = "public"; + protected final String defaultNamespace = defaultTenant + "/default"; + protected int numberOfBookies = 3; + protected final String clusterName = "r1"; + protected URL url; + protected URL urlTls; + protected ServiceConfiguration config = new ServiceConfiguration(); + protected ZookeeperServerTest brokerConfigZk; + protected LocalBookkeeperEnsemble bkEnsemble; + protected PulsarService pulsar; + protected BrokerService broker; + protected PulsarAdmin admin; + protected PulsarClient client; + protected ZooKeeper localZkOfBroker; + protected Object localMetaDataStoreClientCnx; + protected final AtomicBoolean LocalMetadataStoreInReconnectFinishSignal = new AtomicBoolean(); + protected void startZKAndBK() throws Exception { + // Start ZK. + brokerConfigZk = new ZookeeperServerTest(0); + brokerConfigZk.start(); + + // Start BK. + bkEnsemble = new LocalBookkeeperEnsemble(numberOfBookies, 0, () -> 0); + bkEnsemble.start(); + } + + protected void startBrokers() throws Exception { + // Start brokers. + setConfigDefaults(config, clusterName, bkEnsemble, brokerConfigZk); + pulsar = new PulsarService(config); + pulsar.start(); + broker = pulsar.getBrokerService(); + ZKMetadataStore zkMetadataStore = (ZKMetadataStore) pulsar.getLocalMetadataStore(); + localZkOfBroker = zkMetadataStore.getZkClient(); + ClientCnxn cnxn = WhiteboxImpl.getInternalState(localZkOfBroker, "cnxn"); + Object sendThread = WhiteboxImpl.getInternalState(cnxn, "sendThread"); + localMetaDataStoreClientCnx = WhiteboxImpl.getInternalState(sendThread, "clientCnxnSocket"); + + url = new URL(pulsar.getWebServiceAddress()); + urlTls = new URL(pulsar.getWebServiceAddressTls()); + admin = PulsarAdmin.builder().serviceHttpUrl(url.toString()).build(); + client = PulsarClient.builder().serviceUrl(url.toString()).build(); + } + + protected void makeLocalMetadataStoreKeepReconnect() throws Exception { + if (!LocalMetadataStoreInReconnectFinishSignal.compareAndSet(false, true)) { + throw new RuntimeException("Local metadata store is already keeping reconnect"); + } + if (localMetaDataStoreClientCnx.getClass().getSimpleName().equals("ClientCnxnSocketNIO")) { + makeLocalMetadataStoreKeepReconnectNIO(); + } else { + // ClientCnxnSocketNetty. + makeLocalMetadataStoreKeepReconnectNetty(); + } + } + + protected void makeLocalMetadataStoreKeepReconnectNIO() { + new Thread(() -> { + while (LocalMetadataStoreInReconnectFinishSignal.get()) { + try { + SelectionKey sockKey = WhiteboxImpl.getInternalState(localMetaDataStoreClientCnx, "sockKey"); + if (sockKey != null) { + sockKey.channel().close(); + } + // Prevents high cpu usage. + Thread.sleep(5); + } catch (Exception e) { + log.error("Try close the ZK connection of local metadata store failed: {}", e.toString()); + } + } + }).start(); + } + + protected void makeLocalMetadataStoreKeepReconnectNetty() { + new Thread(() -> { + while (LocalMetadataStoreInReconnectFinishSignal.get()) { + try { + Channel channel = WhiteboxImpl.getInternalState(localMetaDataStoreClientCnx, "channel"); + if (channel != null) { + channel.close(); + } + // Prevents high cpu usage. + Thread.sleep(5); + } catch (Exception e) { + log.error("Try close the ZK connection of local metadata store failed: {}", e.toString()); + } + } + }).start(); + } + + protected void stopLocalMetadataStoreAlwaysReconnect() { + LocalMetadataStoreInReconnectFinishSignal.set(false); + } + + protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { + admin.clusters().createCluster(clusterName, ClusterData.builder() + .serviceUrl(url.toString()) + .serviceUrlTls(urlTls.toString()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + + admin.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(clusterName))); + + admin.namespaces().createNamespace(defaultNamespace, Sets.newHashSet(clusterName)); + } + + @Override + protected void setup() throws Exception { + incrementSetupNumber(); + + log.info("--- Starting OneWayReplicatorTestBase::setup ---"); + + startZKAndBK(); + + startBrokers(); + + createDefaultTenantsAndClustersAndNamespace(); + + Thread.sleep(100); + log.info("--- OneWayReplicatorTestBase::setup completed ---"); + } + + private void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setWebServicePort(Optional.of(0)); + config.setWebServicePortTls(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + bookkeeperEnsemble.getZookeeperPort()); + config.setConfigurationMetadataStoreUrl("zk:127.0.0.1:" + brokerConfigZk.getZookeeperPort() + "/foo"); + config.setBrokerDeleteInactiveTopicsEnabled(false); + config.setBrokerDeleteInactiveTopicsFrequencySeconds(60); + config.setBrokerShutdownTimeoutMs(0L); + config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); + config.setBrokerServicePort(Optional.of(0)); + config.setBrokerServicePortTls(Optional.of(0)); + config.setBacklogQuotaCheckIntervalInSeconds(5); + config.setDefaultNumberOfNamespaceBundles(1); + config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + config.setEnableReplicatedSubscriptions(true); + config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + } + + @Override + protected void cleanup() throws Exception { + markCurrentSetupNumberCleaned(); + log.info("--- Shutting down ---"); + + stopLocalMetadataStoreAlwaysReconnect(); + + // Stop brokers. + client.close(); + admin.close(); + if (pulsar != null) { + pulsar.close(); + } + + // Stop ZK and BK. + bkEnsemble.stop(); + brokerConfigZk.stop(); + + // Reset configs. + config = new ServiceConfiguration(); + setConfigDefaults(config, clusterName, bkEnsemble, brokerConfigZk); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java index a2dcf3c9c0b4d..023efc849ca45 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java @@ -230,141 +230,141 @@ public void beforeMethod(Method m) throws Exception { * (11) Restart Broker-1 and connect producer/consumer on cluster-1 * @throws Exception */ - @Test(dataProvider = "TopicsubscriptionTypes") - public void testClusterMigration(boolean persistent, SubscriptionType subType) throws Exception { - log.info("--- Starting ReplicatorTest::testClusterMigration ---"); - final String topicName = BrokerTestUtil - .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespace + "/migrationTopic"); - - @Cleanup - PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) - .build(); - // cluster-1 producer/consumer - Producer producer1 = client1.newProducer().topic(topicName).enableBatching(false) - .producerName("cluster1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - Consumer consumer1 = client1.newConsumer().topic(topicName).subscriptionType(subType) - .subscriptionName("s1").subscribe(); - AbstractTopic topic1 = (AbstractTopic) pulsar1.getBrokerService().getTopic(topicName, false).getNow(null).get(); - retryStrategically((test) -> !topic1.getProducers().isEmpty(), 5, 500); - retryStrategically((test) -> !topic1.getSubscriptions().isEmpty(), 5, 500); - assertFalse(topic1.getProducers().isEmpty()); - assertFalse(topic1.getSubscriptions().isEmpty()); - - // build backlog - consumer1.close(); - int n = 5; - for (int i = 0; i < n; i++) { - producer1.send("test1".getBytes()); - } - - @Cleanup - PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()).statsInterval(0, TimeUnit.SECONDS) - .build(); - // cluster-2 producer/consumer - Producer producer2 = client2.newProducer().topic(topicName).enableBatching(false) - .producerName("cluster2-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - AbstractTopic topic2 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName, false).getNow(null).get(); - assertFalse(topic2.getProducers().isEmpty()); - - ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); - admin1.clusters().updateClusterMigration("r1", true, migratedUrl); - - retryStrategically((test) -> { - try { - topic1.checkClusterMigration().get(); - return true; - } catch (Exception e) { - // ok - } - return false; - }, 10, 500); - - - topic1.checkClusterMigration().get(); - - log.info("before sending message"); - sleep(1000); - producer1.sendAsync("test1".getBytes()); - - // producer is disconnected from cluster-1 - retryStrategically((test) -> topic1.getProducers().isEmpty(), 10, 500); - log.info("before asserting"); - assertTrue(topic1.getProducers().isEmpty()); - - // create 3rd producer on cluster-1 which should be redirected to cluster-2 - Producer producer3 = client1.newProducer().topic(topicName).enableBatching(false) - .producerName("cluster1-2").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - - // producer is connected with cluster-2 - retryStrategically((test) -> topic2.getProducers().size() == 3, 10, 500); - assertTrue(topic2.getProducers().size() == 3); - - // try to consume backlog messages from cluster-1 - consumer1 = client1.newConsumer().topic(topicName).subscriptionName("s1").subscribe(); - if (persistent) { - for (int i = 0; i < n; i++) { - Message msg = consumer1.receive(); - assertEquals(msg.getData(), "test1".getBytes()); - consumer1.acknowledge(msg); - } - } - // after consuming all messages, consumer should have disconnected - // from cluster-1 and reconnect with cluster-2 - retryStrategically((test) -> !topic2.getSubscriptions().isEmpty(), 10, 500); - assertFalse(topic2.getSubscriptions().isEmpty()); - - // not also create a new consumer which should also reconnect to cluster-2 - Consumer consumer2 = client1.newConsumer().topic(topicName).subscriptionType(subType) - .subscriptionName("s2").subscribe(); - retryStrategically((test) -> topic2.getSubscription("s2") != null, 10, 500); - assertFalse(topic2.getSubscription("s2").getConsumers().isEmpty()); - - // publish messages to cluster-2 and consume them - for (int i = 0; i < n; i++) { - producer1.send("test2".getBytes()); - producer2.send("test2".getBytes()); - producer3.send("test2".getBytes()); - } - log.info("Successfully published messages by migrated producers"); - for (int i = 0; i < n * 3; i++) { - assertEquals(consumer1.receive(2, TimeUnit.SECONDS).getData(), "test2".getBytes()); - assertEquals(consumer2.receive(2, TimeUnit.SECONDS).getData(), "test2".getBytes()); - - } - - // create non-migrated topic which should connect to cluster-1 - String diffTopic = BrokerTestUtil - .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespace + "/migrationTopic"); - Consumer consumerDiff = client1.newConsumer().topic(diffTopic).subscriptionType(subType) - .subscriptionName("s1-d").subscribe(); - Producer producerDiff = client1.newProducer().topic(diffTopic).enableBatching(false) - .producerName("cluster1-d").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - AbstractTopic topicDiff = (AbstractTopic) pulsar1.getBrokerService().getTopic(diffTopic, false).getNow(null).get(); - assertNotNull(topicDiff); - for (int i = 0; i < n; i++) { - producerDiff.send("diff".getBytes()); - assertEquals(consumerDiff.receive(2, TimeUnit.SECONDS).getData(), "diff".getBytes()); - } - - // restart broker-1 - broker1.restart(); - Producer producer4 = client1.newProducer().topic(topicName).enableBatching(false) - .producerName("cluster1-4").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - Consumer consumer3 = client1.newConsumer().topic(topicName).subscriptionType(subType) - .subscriptionName("s3").subscribe(); - retryStrategically((test) -> topic2.getProducers().size() == 4, 10, 500); - assertTrue(topic2.getProducers().size() == 4); - retryStrategically((test) -> topic2.getSubscription("s3") != null, 10, 500); - assertFalse(topic2.getSubscription("s3").getConsumers().isEmpty()); - for (int i = 0; i < n; i++) { - producer4.send("test3".getBytes()); - assertEquals(consumer1.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); - assertEquals(consumer2.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); - assertEquals(consumer3.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); - } - - log.info("Successfully consumed messages by migrated consumers"); - } +// @Test(dataProvider = "TopicsubscriptionTypes") +// public void testClusterMigration(boolean persistent, SubscriptionType subType) throws Exception { +// log.info("--- Starting ReplicatorTest::testClusterMigration ---"); +// final String topicName = BrokerTestUtil +// .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespace + "/migrationTopic"); +// +// @Cleanup +// PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) +// .build(); +// // cluster-1 producer/consumer +// Producer producer1 = client1.newProducer().topic(topicName).enableBatching(false) +// .producerName("cluster1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); +// Consumer consumer1 = client1.newConsumer().topic(topicName).subscriptionType(subType) +// .subscriptionName("s1").subscribe(); +// AbstractTopic topic1 = (AbstractTopic) pulsar1.getBrokerService().getTopic(topicName, false).getNow(null).get(); +// retryStrategically((test) -> !topic1.getProducers().isEmpty(), 5, 500); +// retryStrategically((test) -> !topic1.getSubscriptions().isEmpty(), 5, 500); +// assertFalse(topic1.getProducers().isEmpty()); +// assertFalse(topic1.getSubscriptions().isEmpty()); +// +// // build backlog +// consumer1.close(); +// int n = 5; +// for (int i = 0; i < n; i++) { +// producer1.send("test1".getBytes()); +// } +// +// @Cleanup +// PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()).statsInterval(0, TimeUnit.SECONDS) +// .build(); +// // cluster-2 producer/consumer +// Producer producer2 = client2.newProducer().topic(topicName).enableBatching(false) +// .producerName("cluster2-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); +// AbstractTopic topic2 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName, false).getNow(null).get(); +// assertFalse(topic2.getProducers().isEmpty()); +// +// ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); +// admin1.clusters().updateClusterMigration("r1", true, migratedUrl); +// +// retryStrategically((test) -> { +// try { +// topic1.checkClusterMigration().get(); +// return true; +// } catch (Exception e) { +// // ok +// } +// return false; +// }, 10, 500); +// +// +// topic1.checkClusterMigration().get(); +// +// log.info("before sending message"); +// sleep(1000); +// producer1.sendAsync("test1".getBytes()); +// +// // producer is disconnected from cluster-1 +// retryStrategically((test) -> topic1.getProducers().isEmpty(), 10, 500); +// log.info("before asserting"); +// assertTrue(topic1.getProducers().isEmpty()); +// +// // create 3rd producer on cluster-1 which should be redirected to cluster-2 +// Producer producer3 = client1.newProducer().topic(topicName).enableBatching(false) +// .producerName("cluster1-2").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); +// +// // producer is connected with cluster-2 +// retryStrategically((test) -> topic2.getProducers().size() == 3, 10, 500); +// assertTrue(topic2.getProducers().size() == 3); +// +// // try to consume backlog messages from cluster-1 +// consumer1 = client1.newConsumer().topic(topicName).subscriptionName("s1").subscribe(); +// if (persistent) { +// for (int i = 0; i < n; i++) { +// Message msg = consumer1.receive(); +// assertEquals(msg.getData(), "test1".getBytes()); +// consumer1.acknowledge(msg); +// } +// } +// // after consuming all messages, consumer should have disconnected +// // from cluster-1 and reconnect with cluster-2 +// retryStrategically((test) -> !topic2.getSubscriptions().isEmpty(), 10, 500); +// assertFalse(topic2.getSubscriptions().isEmpty()); +// +// // not also create a new consumer which should also reconnect to cluster-2 +// Consumer consumer2 = client1.newConsumer().topic(topicName).subscriptionType(subType) +// .subscriptionName("s2").subscribe(); +// retryStrategically((test) -> topic2.getSubscription("s2") != null, 10, 500); +// assertFalse(topic2.getSubscription("s2").getConsumers().isEmpty()); +// +// // publish messages to cluster-2 and consume them +// for (int i = 0; i < n; i++) { +// producer1.send("test2".getBytes()); +// producer2.send("test2".getBytes()); +// producer3.send("test2".getBytes()); +// } +// log.info("Successfully published messages by migrated producers"); +// for (int i = 0; i < n * 3; i++) { +// assertEquals(consumer1.receive(2, TimeUnit.SECONDS).getData(), "test2".getBytes()); +// assertEquals(consumer2.receive(2, TimeUnit.SECONDS).getData(), "test2".getBytes()); +// +// } +// +// // create non-migrated topic which should connect to cluster-1 +// String diffTopic = BrokerTestUtil +// .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespace + "/migrationTopic"); +// Consumer consumerDiff = client1.newConsumer().topic(diffTopic).subscriptionType(subType) +// .subscriptionName("s1-d").subscribe(); +// Producer producerDiff = client1.newProducer().topic(diffTopic).enableBatching(false) +// .producerName("cluster1-d").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); +// AbstractTopic topicDiff = (AbstractTopic) pulsar1.getBrokerService().getTopic(diffTopic, false).getNow(null).get(); +// assertNotNull(topicDiff); +// for (int i = 0; i < n; i++) { +// producerDiff.send("diff".getBytes()); +// assertEquals(consumerDiff.receive(2, TimeUnit.SECONDS).getData(), "diff".getBytes()); +// } +// +// // restart broker-1 +// broker1.restart(); +// Producer producer4 = client1.newProducer().topic(topicName).enableBatching(false) +// .producerName("cluster1-4").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); +// Consumer consumer3 = client1.newConsumer().topic(topicName).subscriptionType(subType) +// .subscriptionName("s3").subscribe(); +// retryStrategically((test) -> topic2.getProducers().size() == 4, 10, 500); +// assertTrue(topic2.getProducers().size() == 4); +// retryStrategically((test) -> topic2.getSubscription("s3") != null, 10, 500); +// assertFalse(topic2.getSubscription("s3").getConsumers().isEmpty()); +// for (int i = 0; i < n; i++) { +// producer4.send("test3".getBytes()); +// assertEquals(consumer1.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); +// assertEquals(consumer2.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); +// assertEquals(consumer3.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); +// } +// +// log.info("Successfully consumed messages by migrated consumers"); +// } @Test(dataProvider = "TopicsubscriptionTypes") public void testClusterMigrationWithReplicationBacklog(boolean persistent, SubscriptionType subType) throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java index 80db4c30f454d..30867dd2cb44d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java @@ -97,11 +97,13 @@ public void TestConsumedLedgersTrim() throws Exception { } ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); - Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2); + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(managedLedger.getLedgersInfoAsList().size() - 1, msgNum / 2); + }); //no traffic, unconsumed ledger will be retained Thread.sleep(1200); - Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2); + Assert.assertEquals(managedLedger.getLedgersInfoAsList().size() - 1, msgNum / 2); for (int i = 0; i < msgNum; i++) { Message msg = consumer.receive(2, TimeUnit.SECONDS); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java new file mode 100644 index 0000000000000..b06c7db1439a3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java @@ -0,0 +1,163 @@ +/* + * 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 org.apache.pulsar.broker.service; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.MessageDeduplication; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class DeduplicationDisabledBrokerLevelTest extends ProducerConsumerBase { + + private int deduplicationSnapshotFrequency = 5; + private int brokerDeduplicationEntriesInterval = 1000; + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + protected void doInitConf() throws Exception { + this.conf.setBrokerDeduplicationEnabled(false); + this.conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(deduplicationSnapshotFrequency); + this.conf.setBrokerDeduplicationEntriesInterval(brokerDeduplicationEntriesInterval); + } + + @Test + public void testNoBacklogOnDeduplication() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + // deduplication enabled: + // broker level: "false" + // topic level: "true". + // So it is enabled. + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + + // Verify: regarding deduplication cursor, messages will be acknowledged automatically. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("1"); + producer.send("2"); + producer.send("3"); + producer.close(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + Awaitility.await().atMost(Duration.ofSeconds(deduplicationSnapshotFrequency * 3)).untilAsserted(() -> { + PositionImpl LAC = (PositionImpl) ml.getLastConfirmedEntry(); + PositionImpl cursorMD = (PositionImpl) cursor.getMarkDeletedPosition(); + assertTrue(LAC.compareTo(cursorMD) <= 0); + }); + + // cleanup. + admin.topics().delete(topic); + } + + @Test + public void testSnapshotCounterAfterUnload() throws Exception { + final int originalDeduplicationSnapshotFrequency = deduplicationSnapshotFrequency; + deduplicationSnapshotFrequency = 3600; + cleanup(); + setup(); + + // Create a topic and wait deduplication is started. + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic1 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml1 = (ManagedLedgerImpl) persistentTopic1.getManagedLedger(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor1 = + (ManagedCursorImpl) ml1.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor1); + }); + final MessageDeduplication deduplication1 = persistentTopic1.getMessageDeduplication(); + + // 1. Send 999 messages, it is less than "brokerDeduplicationEntriesIntervaddl". + // 2. Unload topic. + // 3. Send 1 messages, there are 1099 messages have not been snapshot now. + // 4. Verify the snapshot has been taken. + // step 1. + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + for (int i = 0; i < brokerDeduplicationEntriesInterval - 1; i++) { + producer.send(i + ""); + } + int snapshotCounter1 = WhiteboxImpl.getInternalState(deduplication1, "snapshotCounter"); + assertEquals(snapshotCounter1, brokerDeduplicationEntriesInterval - 1); + admin.topics().unload(topic); + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) persistentTopic2.getManagedLedger(); + MessageDeduplication deduplication2 = persistentTopic2.getMessageDeduplication(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml2.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + // step 3. + producer.send("last message"); + ml2.trimConsumedLedgersInBackground(new CompletableFuture<>()); + // step 4. + Awaitility.await().untilAsserted(() -> { + int snapshotCounter3 = WhiteboxImpl.getInternalState(deduplication2, "snapshotCounter"); + assertTrue(snapshotCounter3 < brokerDeduplicationEntriesInterval); + // Since https://github.com/apache/pulsar/pull/22034 has not been cherry-pick into branch-3.0, there + // should be 2 ledgers. + // Verify: the previous ledger will be removed because all messages have been acked. + assertEquals(ml2.getLedgersInfo().size(), 1); + }); + + // cleanup. + producer.close(); + admin.topics().delete(topic); + deduplicationSnapshotFrequency = originalDeduplicationSnapshotFrequency; + cleanup(); + setup(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index ac5ab94c213c5..c143cf52b240a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; @@ -33,10 +32,8 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; - import io.netty.buffer.ByteBuf; import io.netty.buffer.UnpooledByteBufAllocator; - import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashSet; @@ -46,7 +43,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; - +import java.util.concurrent.atomic.AtomicReference; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.MediaType; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -59,10 +58,10 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor; import org.apache.pulsar.broker.service.persistent.PersistentMessageFinder; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; -import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ResetCursorData; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; @@ -72,11 +71,10 @@ import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.Commands; import org.awaitility.Awaitility; +import org.mockito.Mockito; +import org.testng.Assert; import org.testng.annotations.Test; -import javax.ws.rs.client.Entity; -import javax.ws.rs.core.MediaType; - @Test(groups = "broker") public class PersistentMessageFinderTest extends MockedBookKeeperTestCase { @@ -234,11 +232,7 @@ public void findEntryFailed(ManagedLedgerException exception, Optional }); assertTrue(ex.get()); - PersistentTopic mock = mock(PersistentTopic.class); - when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); - - PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); + PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor("topicname", c1.getName(), c1, null); monitor.findEntryFailed(new ManagedLedgerException.ConcurrentFindCursorPositionException("failed"), Optional.empty(), null); Field field = monitor.getClass().getDeclaredField("expirationCheckInProgress"); @@ -383,7 +377,7 @@ public static Set getBrokerEntryMetadataIntercep * * @throws Exception */ - @Test(groups = "flaky") + @Test void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithNonRecoverableLedgers"; @@ -402,11 +396,15 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { for (int i = 0; i < totalEntries; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i)); } + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); List ledgers = ledger.getLedgersInfoAsList(); LedgerInfo lastLedgerInfo = ledgers.get(ledgers.size() - 1); - - assertEquals(ledgers.size(), totalEntries / entriesPerLedger); + // The `lastLedgerInfo` should be newly opened, and it does not contain any entries. + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(lastLedgerInfo.getEntries(), 0); + assertEquals(ledgers.size(), totalEntries / entriesPerLedger + 1); // this will make sure that all entries should be deleted Thread.sleep(TimeUnit.SECONDS.toMillis(ttlSeconds)); @@ -415,24 +413,14 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { bkc.deleteLedger(ledgers.get(1).getLedgerId()); bkc.deleteLedger(ledgers.get(2).getLedgerId()); - PersistentTopic mock = mock(PersistentTopic.class); - when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); - - PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); - Position previousMarkDelete = null; - for (int i = 0; i < totalEntries; i++) { - monitor.expireMessages(1); - Position previousPos = previousMarkDelete; - retryStrategically( - (test) -> c1.getMarkDeletedPosition() != null && !c1.getMarkDeletedPosition().equals(previousPos), - 5, 100); - previousMarkDelete = c1.getMarkDeletedPosition(); - } - - PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); - assertEquals(lastLedgerInfo.getLedgerId(), markDeletePosition.getLedgerId()); - assertEquals(lastLedgerInfo.getEntries() - 1, markDeletePosition.getEntryId()); + PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor("topicname", c1.getName(), c1, null); + assertTrue(monitor.expireMessages(ttlSeconds)); + Awaitility.await().untilAsserted(() -> { + PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); + // The markDeletePosition points to the last entry of the previous ledger in lastLedgerInfo. + assertEquals(markDeletePosition.getLedgerId(), lastLedgerInfo.getLedgerId() - 1); + assertEquals(markDeletePosition.getEntryId(), entriesPerLedger - 1); + }); c1.close(); ledger.close(); @@ -440,27 +428,66 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { } - @Test(groups = "flaky") + @Test public void testIncorrectClientClock() throws Exception { final String ledgerAndCursorName = "testIncorrectClientClock"; int maxTTLSeconds = 1; + int entriesNum = 10; ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); // set client clock to 10 days later long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < entriesNum; i++) { + ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp)); + } + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); + // The number of ledgers should be (entriesNum / MaxEntriesPerLedger) + 1 + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(ledger.getLedgersInfoAsList().size(), entriesNum + 1); + PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor("topicname", c1.getName(), c1, null); + Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds)); + monitor.expireMessages(maxTTLSeconds); + assertEquals(c1.getNumberOfEntriesInBacklog(true), 0); + } + + @Test + public void testCheckExpiryByLedgerClosureTimeWithAckUnclosedLedger() throws Throwable { + final String ledgerAndCursorName = "testCheckExpiryByLedgerClosureTimeWithAckUnclosedLedger"; + int maxTTLSeconds = 1; + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(5); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); + // set client clock to 10 days later + long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10); + for (int i = 0; i < 7; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp)); } - assertEquals(ledger.getLedgersInfoAsList().size(), 10); - PersistentTopic mock = mock(PersistentTopic.class); - when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); - PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); + assertEquals(ledger.getLedgersInfoAsList().size(), 2); + PersistentMessageExpiryMonitor monitor = + new PersistentMessageExpiryMonitor("topicname", c1.getName(), c1, null); + AsyncCallbacks.MarkDeleteCallback markDeleteCallback = + (AsyncCallbacks.MarkDeleteCallback) spy( + FieldUtils.readDeclaredField(monitor, "markDeleteCallback", true)); + FieldUtils.writeField(monitor, "markDeleteCallback", markDeleteCallback, true); + + AtomicReference throwableAtomicReference = new AtomicReference<>(); + Mockito.doAnswer(invocation -> { + ManagedLedgerException argument = invocation.getArgument(0, ManagedLedgerException.class); + throwableAtomicReference.set(argument); + return invocation.callRealMethod(); + }).when(markDeleteCallback).markDeleteFailed(any(), any()); + + PositionImpl position = (PositionImpl) ledger.getLastConfirmedEntry(); + c1.markDelete(position); Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds)); monitor.expireMessages(maxTTLSeconds); assertEquals(c1.getNumberOfEntriesInBacklog(true), 0); + + Assert.assertNull(throwableAtomicReference.get()); } @Test @@ -479,16 +506,15 @@ void testMessageExpiryWithPosition() throws Exception { ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); PersistentSubscription subscription = mock(PersistentSubscription.class); - PersistentTopic topic = mock(PersistentTopic.class); + Topic topic = mock(Topic.class); when(subscription.getTopic()).thenReturn(topic); - when(topic.getName()).thenReturn("topicname"); for (int i = 0; i < totalEntries; i++) { positions.add(ledger.addEntry(createMessageWrittenToLedger("msg" + i))); } when(topic.getLastPosition()).thenReturn(positions.get(positions.size() - 1)); - PersistentMessageExpiryMonitor monitor = spy(new PersistentMessageExpiryMonitor(topic, + PersistentMessageExpiryMonitor monitor = spy(new PersistentMessageExpiryMonitor("topicname", cursor.getName(), cursor, subscription)); assertEquals(cursor.getMarkDeletedPosition(), PositionImpl.get(positions.get(0).getLedgerId(), -1)); boolean issued; @@ -527,7 +553,7 @@ void testMessageExpiryWithPosition() throws Exception { clearInvocations(monitor); ManagedCursorImpl mockCursor = mock(ManagedCursorImpl.class); - PersistentMessageExpiryMonitor mockMonitor = spy(new PersistentMessageExpiryMonitor(topic, + PersistentMessageExpiryMonitor mockMonitor = spy(new PersistentMessageExpiryMonitor("topicname", cursor.getName(), mockCursor, subscription)); // Not calling findEntryComplete to clear expirationCheckInProgress condition, so following call to // expire message shouldn't issue. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index 71310fef8102a..2f7455cbabba8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -205,6 +205,7 @@ public void setup() throws Exception { doReturn(spy(DefaultEventLoop.class)).when(channel).eventLoop(); doReturn(channel).when(ctx).channel(); doReturn(ctx).when(serverCnx).ctx(); + doReturn(CompletableFuture.completedFuture(true)).when(serverCnx).checkConnectionLiveness(); NamespaceService nsSvc = mock(NamespaceService.class); NamespaceBundle bundle = mock(NamespaceBundle.class); @@ -685,7 +686,15 @@ public void testSubscribeUnsubscribe() throws Exception { f1.get(); // 2. duplicate subscribe - Future f2 = topic.subscribe(getSubscriptionOption(cmd)); + CommandSubscribe cmd2 = new CommandSubscribe() + .setConsumerId(2) + .setTopic(successTopicName) + .setSubscription(successSubName) + .setConsumerName("consumer-name") + .setReadCompacted(false) + .setRequestId(2) + .setSubType(SubType.Exclusive); + Future f2 = topic.subscribe(getSubscriptionOption(cmd2)); try { f2.get(); fail("should fail with exception"); @@ -750,19 +759,11 @@ public void testAddRemoveConsumer() throws Exception { sub.addConsumer(consumer); assertTrue(sub.getDispatcher().isConsumerConnected()); - // 2. duplicate add consumer - try { - sub.addConsumer(consumer).get(); - fail("Should fail with ConsumerBusyException"); - } catch (Exception e) { - assertTrue(e.getCause() instanceof BrokerServiceException.ConsumerBusyException); - } - - // 3. simple remove consumer + // 2. simple remove consumer sub.removeConsumer(consumer); assertFalse(sub.getDispatcher().isConsumerConnected()); - // 4. duplicate remove consumer + // 3. duplicate remove consumer try { sub.removeConsumer(consumer); fail("Should fail with ServerMetadataException"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 159d49ca2e7cc..3fa3a8dfc0f15 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -727,36 +727,6 @@ public void testReplicatorClearBacklog() throws Exception { assertEquals(status.getReplicationBacklog(), 0); } - - @Test(timeOut = 30000) - public void testResetReplicatorSubscriptionPosition() throws Exception { - final TopicName dest = TopicName - .get(BrokerTestUtil.newUniqueName("persistent://pulsar/ns/resetReplicatorSubscription")); - - @Cleanup - MessageProducer producer1 = new MessageProducer(url1, dest); - - // Produce from cluster1 and consume from the rest - for (int i = 0; i < 10; i++) { - producer1.produce(2); - } - - PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(dest.toString()).get(); - - PersistentReplicator replicator = (PersistentReplicator) spy( - topic.getReplicators().get(topic.getReplicators().keys().get(0))); - - MessageId id = topic.getLastMessageId().get(); - admin1.topics().expireMessages(dest.getPartitionedTopicName(), - replicator.getCursor().getName(), - id,false); - - replicator.updateRates(); - - ReplicatorStats status = replicator.getStats(); - assertEquals(status.getReplicationBacklog(), 0); - } - @Test(timeOut = 30000) public void testResetCursorNotFail() throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 8abd6dcff8de4..202514f3e85b4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -71,6 +71,7 @@ import java.util.function.Predicate; import java.util.function.Supplier; import lombok.AllArgsConstructor; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; @@ -104,6 +105,7 @@ import org.apache.pulsar.broker.service.utils.ClientChannelHelper; import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.util.ConsumerName; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.AuthMethod; import org.apache.pulsar.common.api.proto.BaseCommand; @@ -1070,6 +1072,98 @@ public void testProducerChangeSocket() throws Exception { channel2.close(); } + @Test + public void testHandleConsumerAfterClientChannelInactive() throws Exception { + final String tName = successTopicName; + final long consumerId = 1; + final MutableInt requestId = new MutableInt(1); + final String sName = successSubName; + final String cName1 = ConsumerName.generateRandomName(); + final String cName2 = ConsumerName.generateRandomName(); + resetChannel(); + setChannelConnected(); + + // The producer register using the first connection. + ByteBuf cmdSubscribe1 = Commands.newSubscribe(tName, sName, consumerId, requestId.incrementAndGet(), + SubType.Exclusive, 0, cName1, 0); + channel.writeInbound(cmdSubscribe1); + assertTrue(getResponse() instanceof CommandSuccess); + PersistentTopic topicRef = (PersistentTopic) brokerService.getTopicReference(tName).get(); + assertNotNull(topicRef); + assertNotNull(topicRef.getSubscription(sName).getConsumers()); + assertEquals(topicRef.getSubscription(sName).getConsumers().size(), 1); + assertEquals(topicRef.getSubscription(sName).getConsumers().iterator().next().consumerName(), cName1); + + // Verify the second producer using a new connection will override the consumer who using a stopped channel. + channelsStoppedAnswerHealthCheck.add(channel); + ClientChannel channel2 = new ClientChannel(); + setChannelConnected(channel2.serverCnx); + ByteBuf cmdSubscribe2 = Commands.newSubscribe(tName, sName, consumerId, requestId.incrementAndGet(), + CommandSubscribe.SubType.Exclusive, 0, cName2, 0); + channel2.channel.writeInbound(cmdSubscribe2); + BackGroundExecutor backGroundExecutor = startBackgroundExecutorForEmbeddedChannel(channel); + + assertTrue(getResponse(channel2.channel, channel2.clientChannelHelper) instanceof CommandSuccess); + assertEquals(topicRef.getSubscription(sName).getConsumers().size(), 1); + assertEquals(topicRef.getSubscription(sName).getConsumers().iterator().next().consumerName(), cName2); + backGroundExecutor.close(); + + // cleanup. + channel.finish(); + channel2.close(); + } + + @Test + public void test2ndSubFailedIfDisabledConCheck() + throws Exception { + final String tName = successTopicName; + final long consumerId = 1; + final MutableInt requestId = new MutableInt(1); + final String sName = successSubName; + final String cName1 = ConsumerName.generateRandomName(); + final String cName2 = ConsumerName.generateRandomName(); + // Disabled connection check. + pulsar.getConfig().setConnectionLivenessCheckTimeoutMillis(-1); + resetChannel(); + setChannelConnected(); + + // The consumer register using the first connection. + ByteBuf cmdSubscribe1 = Commands.newSubscribe(tName, sName, consumerId, requestId.incrementAndGet(), + SubType.Exclusive, 0, cName1, 0); + channel.writeInbound(cmdSubscribe1); + assertTrue(getResponse() instanceof CommandSuccess); + PersistentTopic topicRef = (PersistentTopic) brokerService.getTopicReference(tName).orElse(null); + assertNotNull(topicRef); + assertNotNull(topicRef.getSubscription(sName).getConsumers()); + assertEquals(topicRef.getSubscription(sName).getConsumers().stream().map(Consumer::consumerName) + .collect(Collectors.toList()), Collections.singletonList(cName1)); + + // Verify the consumer using a new connection will override the consumer who using a stopped channel. + channelsStoppedAnswerHealthCheck.add(channel); + ClientChannel channel2 = new ClientChannel(); + setChannelConnected(channel2.serverCnx); + ByteBuf cmdSubscribe2 = Commands.newSubscribe(tName, sName, consumerId, requestId.incrementAndGet(), + CommandSubscribe.SubType.Exclusive, 0, cName2, 0); + channel2.channel.writeInbound(cmdSubscribe2); + BackGroundExecutor backGroundExecutor = startBackgroundExecutorForEmbeddedChannel(channel); + + // Since the feature "ConnectionLiveness" has been disabled, the fix + // by https://github.com/apache/pulsar/pull/21183 will not be affected, so the client will still get an error. + Object responseOfConnection2 = getResponse(channel2.channel, channel2.clientChannelHelper); + assertTrue(responseOfConnection2 instanceof CommandError); + assertTrue(((CommandError) responseOfConnection2).getMessage() + .contains("Exclusive consumer is already connected")); + assertEquals(topicRef.getSubscription(sName).getConsumers().size(), 1); + assertEquals(topicRef.getSubscription(sName).getConsumers().iterator().next().consumerName(), cName1); + backGroundExecutor.close(); + + // cleanup. + channel.finish(); + channel2.close(); + // Reset configuration. + pulsar.getConfig().setConnectionLivenessCheckTimeoutMillis(5000); + } + /** * When a channel typed "EmbeddedChannel", once we call channel.execute(runnable), there is no background thread * to run it. @@ -1913,9 +2007,11 @@ public void testDuplicateConcurrentSubscribeCommand() throws Exception { "test" /* consumer name */, 0 /* avoid reseting cursor */); channel.writeInbound(clientCommand); + BackGroundExecutor backGroundExecutor = startBackgroundExecutorForEmbeddedChannel(channel); + // Create producer second time clientCommand = Commands.newSubscribe(successTopicName, // - successSubName, 2 /* consumer id */, 1 /* request id */, SubType.Exclusive, 0, + successSubName, 2 /* consumer id */, 2 /* request id */, SubType.Exclusive, 0, "test" /* consumer name */, 0 /* avoid reseting cursor */); channel.writeInbound(clientCommand); @@ -1925,6 +2021,9 @@ public void testDuplicateConcurrentSubscribeCommand() throws Exception { CommandError error = (CommandError) response; assertEquals(error.getError(), ServerError.ConsumerBusy); }); + + // cleanup. + backGroundExecutor.close(); channel.finish(); } @@ -2779,13 +2878,7 @@ protected Object getResponse(EmbeddedChannel channel, ClientChannelHelper client if (channelsStoppedAnswerHealthCheck.contains(channel)) { continue; } - channel.writeAndFlush(Commands.newPong()).addListener(future -> { - if (!future.isSuccess()) { - log.warn("[{}] Forcing connection to close since cannot send a pong message.", - channel, future.cause()); - channel.close(); - } - }); + channel.writeInbound(Commands.newPong()); continue; } return cmd; @@ -3287,8 +3380,9 @@ public boolean isCompletedExceptionally() { }; // assert error response assertTrue(responseAssert.test(responseAssert)); - // assert consumer-delete event occur - assertEquals(1L, + // The delete event will only occur after the future is completed. + // assert consumer-delete event will not occur. + assertEquals(0L, deleteTimesMark.getAllValues().stream().filter(f -> f == existingConsumerFuture).count()); // Server will not close the connection assertTrue(channel.isOpen()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index bf2698d400872..16eda4e86c015 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -72,6 +72,8 @@ public class SystemTopicBasedTopicPoliciesServiceTest extends MockedPulsarServic private static final String NAMESPACE4 = "system-topic/namespace-4"; + private static final String NAMESPACE5 = "system-topic/namespace-5"; + private static final TopicName TOPIC1 = TopicName.get("persistent", NamespaceName.get(NAMESPACE1), "topic-1"); private static final TopicName TOPIC2 = TopicName.get("persistent", NamespaceName.get(NAMESPACE1), "topic-2"); private static final TopicName TOPIC3 = TopicName.get("persistent", NamespaceName.get(NAMESPACE2), "topic-1"); @@ -465,4 +467,21 @@ public void testWriterCache() throws Exception { admin.namespaces().deleteNamespace(NAMESPACE4); Assert.assertNull(service.getWriterCaches().synchronous().getIfPresent(NamespaceName.get(NAMESPACE4))); } + + @Test + public void testPrepareInitPoliciesCacheAsyncWhenNamespaceBeingDeleted() throws Exception { + SystemTopicBasedTopicPoliciesService service = (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService(); + admin.namespaces().createNamespace(NAMESPACE5); + + NamespaceName namespaceName = NamespaceName.get(NAMESPACE5); + pulsar.getPulsarResources().getNamespaceResources().setPolicies(namespaceName, + old -> { + old.deleted = true; + return old; + }); + + assertNull(service.getPoliciesCacheInit(namespaceName)); + service.prepareInitPoliciesCacheAsync(namespaceName).get(); + admin.namespaces().deleteNamespace(NAMESPACE5); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 54fec3934ddbc..981feb002871e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -38,11 +38,11 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -217,7 +217,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { Thread.sleep(2000); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); @@ -303,7 +303,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { assertEquals(opLatencyMetricsSum.intValue(), opLatencyTopicMetrics.get().value); ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, true, true, namespaceOutput); + PrometheusMetricsTestUtil.generate(pulsar, false, true, true, namespaceOutput); Multimap namespaceMetricsMap = PrometheusMetricsTest.parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); Optional namespaceMetric = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java new file mode 100644 index 0000000000000..f24c5c5933e5b --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -0,0 +1,101 @@ +/* + * 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 org.apache.pulsar.broker.service.persistent; + +import com.carrotsearch.hppc.ObjectSet; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class PersistentDispatcherMultipleConsumersTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + List consumerList = + WhiteboxImpl.getInternalState(dispatcher, "consumerList"); + + org.apache.pulsar.broker.service.Consumer serviceConsumer = consumerList.get(0); + consumerSet.add(serviceConsumer); + consumerList.add(serviceConsumer); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList2() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + consumerSet.clear(); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 717dfc28ac884..5c49b4723033b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -57,13 +57,18 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -74,6 +79,8 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; @@ -106,6 +113,11 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Override protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerCursorBackloggedThreshold(10); + } + /** * Test validates that broker cleans up topic which failed to unload while bundle unloading. * @@ -354,7 +366,7 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex latch.await(10, TimeUnit.SECONDS); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); @@ -661,4 +673,92 @@ public void testDynamicConfigurationAutoSkipNonRecoverableData() throws Exceptio subscribe.close(); admin.topics().delete(topicName); } + + @Test + public void testAddWaitingCursorsForNonDurable() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/testAddWaitingCursors"; + admin.topics().createNonPartitionedTopic(topicName); + final Optional topic = pulsar.getBrokerService().getTopic(topicName, false).join(); + assertNotNull(topic.get()); + PersistentTopic persistentTopic = (PersistentTopic) topic.get(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl)persistentTopic.getManagedLedger(); + final ManagedCursor spyCursor= spy(ledger.newNonDurableCursor(PositionImpl.LATEST, "sub-2")); + doAnswer((invocation) -> { + Thread.sleep(5_000); + invocation.callRealMethod(); + return null; + }).when(spyCursor).asyncReadEntriesOrWait(any(int.class), any(long.class), + any(AsyncCallbacks.ReadEntriesCallback.class), any(Object.class), any(PositionImpl.class)); + Field cursorField = ManagedLedgerImpl.class.getDeclaredField("cursors"); + cursorField.setAccessible(true); + ManagedCursorContainer container = (ManagedCursorContainer) cursorField.get(ledger); + container.removeCursor("sub-2"); + container.add(spyCursor, null); + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionName("sub-2").subscribe(); + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + producer.send("test"); + producer.close(); + final Message receive = consumer.receive(); + assertEquals("test", receive.getValue()); + consumer.close(); + Awaitility.await() + .pollDelay(5, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertEquals(ledger.getWaitingCursorsCount(), 0); + }); + } + + @Test + public void testAddWaitingCursorsForNonDurable2() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/testAddWaitingCursors2"; + admin.topics().createNonPartitionedTopic(topicName); + pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.Durable) + .subscriptionType(SubscriptionType.Shared) + .subscriptionName("sub-1").subscribe().close(); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).enableBatching(false).topic(topicName).create(); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionName("sub-2").subscribe(); + int count = 0; + while(true) { + final Message msg = consumer.receive(3, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 100); + Thread.sleep(3_000); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + while(true) { + final Message msg = consumer.receive(5, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 200); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index a7d33737bdc3c..3623a47d9ef67 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -41,12 +41,12 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; @@ -120,7 +120,7 @@ public void testSchemaRegistryMetrics() throws Exception { deleteSchema(schemaId, version(1)); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index f29c643a8f50b..de65d5db56475 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -43,6 +43,7 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; @@ -50,7 +51,6 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterProducerTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -333,7 +333,7 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel consumer2.updateRates(); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 8ae0242c6232a..7368d42355c1b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -28,10 +28,10 @@ import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -99,7 +99,7 @@ public void testMetadataStoreStats() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); @@ -189,7 +189,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 455da11f4fdf0..15a9a2ca3f83d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -19,7 +19,10 @@ package org.apache.pulsar.broker.stats; import static com.google.common.base.Preconditions.checkArgument; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -35,6 +38,7 @@ import java.math.RoundingMode; import java.nio.charset.StandardCharsets; import java.text.NumberFormat; +import java.time.Clock; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -52,6 +56,7 @@ import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -62,6 +67,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; @@ -84,7 +90,6 @@ import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.apache.zookeeper.CreateMode; import org.awaitility.Awaitility; -import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -167,7 +172,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep }); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_publish_rate_limit_times")); @@ -203,7 +208,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep @Cleanup ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); assertTrue(metrics2.containsKey("pulsar_publish_rate_limit_times")); @@ -235,7 +240,7 @@ public void testMetricsTopicCount() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_topics_count"); @@ -272,7 +277,7 @@ public void testMetricsAvgMsgSize2() throws Exception { producerInServer.getStats().msgThroughputIn = 100; @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_average_msg_size")); @@ -315,7 +320,7 @@ public void testPerTopicStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -413,7 +418,7 @@ public void testPerBrokerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -522,7 +527,7 @@ public void testPerTopicStatsReconnect() throws Exception { c2.close(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -600,7 +605,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric true ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -632,7 +637,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric false ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); @@ -716,7 +721,7 @@ public void testPerTopicExpiredStat() throws Exception { Awaitility.await().until(() -> sub2.getExpiredMessageRate() != 0.0); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); // There should be 2 metrics with different tags for each topic @@ -797,7 +802,7 @@ public void testBundlesMetrics() throws Exception { loadManager.getLoadManager().updateLocalBrokerData(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_in")); @@ -842,7 +847,7 @@ public void testNonPersistentSubMetrics() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_subscription_back_log")); @@ -889,7 +894,7 @@ public void testPerNamespaceStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -962,7 +967,7 @@ public void testPerProducerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1030,7 +1035,7 @@ public void testPerConsumerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, true, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1117,7 +1122,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); @@ -1221,7 +1226,7 @@ public void testManagedLedgerCacheStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1257,7 +1262,7 @@ public void testManagedLedgerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1335,7 +1340,7 @@ public void testManagedLedgerBookieClientStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1416,7 +1421,7 @@ public String getCommandData() { }); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_authentication_success_total"); @@ -1477,7 +1482,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_expired_token_total"); @@ -1518,7 +1523,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); @@ -1592,7 +1597,7 @@ public void testManagedCursorPersistStats() throws Exception { // enable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(true); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1605,7 +1610,7 @@ public void testManagedCursorPersistStats() throws Exception { // disable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(false); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); @@ -1624,7 +1629,7 @@ public void testBrokerConnection() throws Exception { .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_connection_created_total_count"); @@ -1641,7 +1646,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1664,7 +1669,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1708,7 +1713,7 @@ public void testCompaction() throws Exception { .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1743,7 +1748,7 @@ public void testCompaction() throws Exception { Compactor compactor = ((PulsarCompactionServiceFactory)pulsar.getCompactionServiceFactory()).getCompactor(); compactor.compact(topicName).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1776,31 +1781,36 @@ public void testCompaction() throws Exception { @Test public void testMetricsWithCache() throws Throwable { - ServiceConfiguration configuration = Mockito.mock(ServiceConfiguration.class); - Mockito.when(configuration.getManagedLedgerStatsPeriodSeconds()).thenReturn(2); - Mockito.when(configuration.isMetricsBufferResponse()).thenReturn(true); - Mockito.when(configuration.getClusterName()).thenReturn(configClusterName); - Mockito.when(pulsar.getConfiguration()).thenReturn(configuration); + ServiceConfiguration configuration = pulsar.getConfiguration(); + configuration.setManagedLedgerStatsPeriodSeconds(2); + configuration.setMetricsBufferResponse(true); + configuration.setClusterName(configClusterName); - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - TimeWindow timeWindow = new TimeWindow<>(2, (int) TimeUnit.SECONDS.toMillis(period)); + // create a mock clock to control the time + AtomicLong currentTimeMillis = new AtomicLong(System.currentTimeMillis()); + Clock clock = mock(Clock.class); + when(clock.millis()).thenAnswer(invocation -> currentTimeMillis.get()); + PrometheusMetricsGenerator prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, true, false, false, + false, clock); + + String previousMetrics = null; for (int a = 0; a < 4; a++) { - long start = System.currentTimeMillis(); ByteArrayOutputStream statsOut1 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut1, null); + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut1, null); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut2, null); - long end = System.currentTimeMillis(); - - if (timeWindow.currentWindowStart(start) == timeWindow.currentWindowStart(end)) { - String metricsStr1 = statsOut1.toString(); - String metricsStr2 = statsOut2.toString(); - assertEquals(metricsStr1, metricsStr2); - Multimap metrics = parseMetrics(metricsStr1); - } + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut2, null); + + String metricsStr1 = statsOut1.toString(); + String metricsStr2 = statsOut2.toString(); + assertTrue(metricsStr1.length() > 1000); + assertEquals(metricsStr1, metricsStr2); + assertNotEquals(metricsStr1, previousMetrics); + previousMetrics = metricsStr1; - Thread.sleep(TimeUnit.SECONDS.toMillis(period / 2)); + // move time forward + currentTimeMillis.addAndGet(TimeUnit.SECONDS.toMillis(2)); } } @@ -1828,7 +1838,7 @@ public void testSplitTopicAndPartitionLabel() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_consumers_count"); @@ -1864,7 +1874,7 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Pattern typePattern = Pattern.compile("^#\\s+TYPE\\s+(\\w+)\\s+(\\w+)"); @@ -1984,7 +1994,7 @@ public void testEscapeLabelValue() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); final List subCountLines = metricsStr.lines() @@ -1994,7 +2004,7 @@ public void testEscapeLabelValue() throws Exception { assertEquals(subCountLines.size(), 1); assertEquals(subCountLines.get(0), "pulsar_subscription_msg_drop_rate{cluster=\"test\",namespace=\"prop/ns-abc1\"," - + "topic=\"persistent://prop/ns-abc1/\\\"mytopic\",subscription=\"s\\\"ub\\\\\"} 0.0"); + + "topic=\"persistent://prop/ns-abc1/\\\"mytopic\",subscription=\"s\\\"ub\\\"} 0.0"); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index d5e0066a86f15..83e6f43cbafcc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -29,13 +29,13 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -231,7 +231,7 @@ public void testSubscriptionStats(final String topic, final String subName, bool } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, enableTopicStats, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java deleted file mode 100644 index 89528c1965397..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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 org.apache.pulsar.broker.stats; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import org.testng.annotations.Test; - -public class TimeWindowTest { - - @Test - public void windowTest() throws Exception { - int intervalInMs = 1000; - int sampleCount = 2; - TimeWindow timeWindow = new TimeWindow<>(sampleCount, intervalInMs); - - WindowWrap expect1 = timeWindow.current(oldValue -> 1); - WindowWrap expect2 = timeWindow.current(oldValue -> null); - assertNotNull(expect1); - assertNotNull(expect2); - - if (expect1.start() == expect2.start()) { - assertEquals((int) expect1.value(), 1); - assertEquals(expect1, expect2); - assertEquals(expect1.value(), expect2.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect3 = timeWindow.current(oldValue -> 2); - WindowWrap expect4 = timeWindow.current(oldValue -> null); - assertNotNull(expect3); - assertNotNull(expect4); - - if (expect3.start() == expect4.start()) { - assertEquals((int) expect3.value(), 2); - assertEquals(expect3, expect4); - assertEquals(expect3.value(), expect4.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect5 = timeWindow.current(oldValue -> 3); - WindowWrap expect6 = timeWindow.current(oldValue -> null); - assertNotNull(expect5); - assertNotNull(expect6); - - if (expect5.start() == expect6.start()) { - assertEquals((int) expect5.value(), 3); - assertEquals(expect5, expect6); - assertEquals(expect5.value(), expect6.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect7 = timeWindow.current(oldValue -> 4); - WindowWrap expect8 = timeWindow.current(oldValue -> null); - assertNotNull(expect7); - assertNotNull(expect8); - - if (expect7.start() == expect8.start()) { - assertEquals((int) expect7.value(), 4); - assertEquals(expect7, expect8); - assertEquals(expect7.value(), expect8.value()); - } - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 4d38f5fad5141..85c6dd795d759 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -37,9 +37,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -117,7 +117,7 @@ public void testTransactionCoordinatorMetrics() throws Exception { pulsar.getTransactionMetadataStoreService().getStores() .get(transactionCoordinatorIDTwo).newTransaction(timeout, null).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_txn_active_count"); @@ -185,7 +185,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { pulsar.getBrokerService().updateRates(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -215,7 +215,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { }); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -271,7 +271,7 @@ public void testManagedLedgerMetrics() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -289,7 +289,7 @@ public void testManagedLedgerMetrics() throws Exception { checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 126, metric); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -333,7 +333,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -358,7 +358,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -392,7 +392,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { .send(); } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 12f126b20bd7b..2ed12a753a0f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -1505,7 +1505,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { fail("Expect failure by PendingAckHandle closed, but success"); } catch (ExecutionException executionException){ Throwable t = executionException.getCause(); - Assert.assertTrue(t instanceof BrokerServiceException.ServiceUnitNotReadyException); + Assert.assertTrue(t instanceof BrokerServiceException); } } @@ -1653,7 +1653,7 @@ public void testGetTxnState() throws Exception { @Test public void testEncryptionRequired() throws Exception { - final String namespace = "tnx/ns-prechecks"; + final String namespace = "tnx/testEncryptionRequired"; final String topic = "persistent://" + namespace + "/test_transaction_topic"; admin.namespaces().createNamespace(namespace); admin.namespaces().setEncryptionRequiredStatus(namespace, true); @@ -1786,7 +1786,7 @@ private void getTopic(String topicName) { @Test public void testReadCommittedWithReadCompacted() throws Exception{ - final String namespace = "tnx/ns-prechecks"; + final String namespace = "tnx/testReadCommittedWithReadCompacted"; final String topic = "persistent://" + namespace + "/test_transaction_topic"; admin.namespaces().createNamespace(namespace); admin.topics().createNonPartitionedTopic(topic); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index 1dc086dbe3470..0211b860cc4fe 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -36,10 +36,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl; @@ -226,7 +226,7 @@ public void testTransactionBufferMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index bc537fb784f0e..f5ac68950b49d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -21,7 +21,9 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; @@ -42,15 +44,17 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; +import org.apache.pulsar.PrometheusMetricsTestUtil; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; @@ -58,6 +62,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; @@ -99,6 +104,83 @@ protected void cleanup() { super.internalCleanup(); } + /** + * Test consumer can be built successfully with retryable exception + * and get correct error with no-retryable exception. + * @throws Exception + */ + @Test(timeOut = 60000) + public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { + // 1. Prepare and make sure the consumer can be built successfully. + String topic = NAMESPACE1 + "/testUnloadSubscriptionWhenFailedInitPendingAck"; + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer() + .subscriptionName("subName1") + .topic(topic) + .subscribe(); + // 2. Mock a transactionPendingAckStoreProvider to test building consumer + // failing at transactionPendingAckStoreProvider::checkInitializedBefore. + Field transactionPendingAckStoreProviderField = PulsarService.class + .getDeclaredField("transactionPendingAckStoreProvider"); + transactionPendingAckStoreProviderField.setAccessible(true); + TransactionPendingAckStoreProvider pendingAckStoreProvider = + (TransactionPendingAckStoreProvider) transactionPendingAckStoreProviderField + .get(pulsarServiceList.get(0)); + TransactionPendingAckStoreProvider mockProvider = mock(pendingAckStoreProvider.getClass()); + // 3. Test retryable exception when checkInitializedBefore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + // First, the method checkInitializedBefore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail initialize"))) + // Then, the method will be executed successfully. + .thenReturn(CompletableFuture.completedFuture(false)); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer() + .subscriptionName("subName2") + .topic(topic) + .subscribe(); + + // 4. Test retryable exception when newPendingAckStore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + .thenReturn(CompletableFuture.completedFuture(true)); + + when(mockProvider.newPendingAckStore(any())) + // First, the method newPendingAckStore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail new store"))) + // Then, the method will be executed successfully. + .thenCallRealMethod(); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer3 = pulsarClient.newConsumer() + .subscriptionName("subName3") + .topic(topic) + .subscribe(); + + // 5. Test no-retryable exception: + // The consumer building will be failed without retrying. + when(mockProvider.checkInitializedBefore(any())) + // The method checkInitializedBefore will fail with a no-retryable exception without retrying. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException + .NonRecoverableLedgerException("mock fail"))) + .thenReturn(CompletableFuture.completedFuture(false)); + @Cleanup PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(pulsarServiceList.get(0).getBrokerServiceUrl()) + .operationTimeout(3, TimeUnit.SECONDS) + .build(); + try { + @Cleanup + Consumer consumer4 = pulsarClient.newConsumer() + .subscriptionName("subName4") + .topic(topic) + .subscribe(); + fail(); + } catch (Exception exception) { + assertTrue(exception.getMessage().contains("Failed to init transaction pending ack.")); + } + } + @Test public void individualPendingAckReplayTest() throws Exception { int messageCount = 1000; @@ -254,7 +336,7 @@ public void testPendingAckMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index b069d31dc6e0d..999f22dc7f92b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -26,10 +26,12 @@ import com.google.common.io.CharStreams; import com.google.common.io.Closeables; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; +import java.io.BufferedReader; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.net.HttpURLConnection; import java.net.URL; import java.security.KeyStore; import java.security.PrivateKey; @@ -42,6 +44,8 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipException; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.KeyManager; import javax.net.ssl.KeyManagerFactory; @@ -49,10 +53,10 @@ import javax.net.ssl.TrustManager; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -102,7 +106,7 @@ public class WebServiceTest { public void testWebExecutorMetrics() throws Exception { setupEnv(true, false, false, false, -1, false); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); @@ -353,6 +357,71 @@ public void testBrokerReady() throws Exception { assertEquals(res.getResponseBody(), "ok"); } + @Test + public void testCompressOutputMetricsInPrometheus() throws Exception { + setupEnv(true, false, false, false, -1, false); + + String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; + + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); + connection.setRequestProperty("Accept-Encoding", "gzip"); + + StringBuilder content = new StringBuilder(); + + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + // Process the decompressed content + int data; + while ((data = gzipInputStream.read()) != -1) { + content.append((char) data); + } + } + + log.info("Response Content: {}", content); + assertTrue(content.toString().contains("process_cpu_seconds_total")); + } catch (IOException e) { + log.error("Failed to decompress the content, likely the content is not compressed ", e); + fail(); + } finally { + connection.disconnect(); + } + } + + @Test + public void testUnCompressOutputMetricsInPrometheus() throws Exception { + setupEnv(true, false, false, false, -1, false); + + String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; + + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); + + StringBuilder content = new StringBuilder(); + + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + fail(); + } catch (IOException e) { + assertTrue(e instanceof ZipException); + } + + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + String line; + while ((line = reader.readLine()) != null) { + content.append(line + "\n"); + } + } finally { + connection.disconnect(); + } + + log.info("Response Content: {}", content); + + assertTrue(content.toString().contains("process_cpu_seconds_total")); + } + private String makeHttpRequest(boolean useTls, boolean useAuth) throws Exception { InputStream response = null; try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 18fb141be3178..7219555050839 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -38,6 +38,7 @@ import java.util.Optional; import java.util.Random; import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListSet; @@ -48,12 +49,17 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import lombok.Cleanup; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.common.api.proto.KeySharedMode; import org.apache.pulsar.common.naming.TopicDomain; @@ -61,6 +67,7 @@ import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.util.Murmur3_32Hash; import org.awaitility.Awaitility; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -1630,4 +1637,263 @@ public void testContinueDispatchMessagesWhenMessageDelayed() throws Exception { log.info("Got {} other messages...", sum); Assert.assertEquals(sum, delayedMessages + messages); } + + private AtomicInteger injectReplayReadCounter(String topicName, String cursorName) throws Exception { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor(cursorName); + managedLedger.getCursors().removeCursor(cursor.getName()); + managedLedger.getActiveCursors().removeCursor(cursor.getName()); + ManagedCursorImpl spyCursor = Mockito.spy(cursor); + managedLedger.getCursors().add(spyCursor, PositionImpl.EARLIEST); + managedLedger.getActiveCursors().add(spyCursor, PositionImpl.EARLIEST); + AtomicInteger replyReadCounter = new AtomicInteger(); + Mockito.doAnswer(invocation -> { + if (!String.valueOf(invocation.getArguments()[2]).equals("Normal")) { + replyReadCounter.incrementAndGet(); + } + return invocation.callRealMethod(); + }).when(spyCursor).asyncReplayEntries(Mockito.anySet(), Mockito.any(), Mockito.any()); + Mockito.doAnswer(invocation -> { + if (!String.valueOf(invocation.getArguments()[2]).equals("Normal")) { + replyReadCounter.incrementAndGet(); + } + return invocation.callRealMethod(); + }).when(spyCursor).asyncReplayEntries(Mockito.anySet(), Mockito.any(), Mockito.any(), Mockito.anyBoolean()); + admin.topics().createSubscription(topicName, cursorName, MessageId.earliest); + return replyReadCounter; + } + + @Test + public void testNoRepeatedReadAndDiscard() throws Exception { + int delayedMessages = 100; + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "my-sub"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); + + // Send messages. + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); + for (int i = 0; i < delayedMessages; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + producer.close(); + + // Make ack holes. + Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + List msgList1 = new ArrayList<>(); + List msgList2 = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg1 = consumer1.receive(1, TimeUnit.SECONDS); + if (msg1 != null) { + msgList1.add(msg1); + } + Message msg2 = consumer2.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + msgList2.add(msg2); + } + } + Consumer redeliverConsumer = null; + if (!msgList1.isEmpty()) { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + redeliverConsumer = consumer2; + } else { + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + redeliverConsumer = consumer1; + } + + // consumer3 will be added to the "recentJoinedConsumers". + Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(1000) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + redeliverConsumer.close(); + + // Verify: no repeated Read-and-discard. + Thread.sleep(5 * 1000); + int maxReplayCount = delayedMessages * 2; + log.info("Reply read count: {}", replyReadCounter.get()); + assertTrue(replyReadCounter.get() < maxReplayCount); + + // cleanup. + consumer1.close(); + consumer2.close(); + consumer3.close(); + admin.topics().delete(topic, false); + } + + /** + * This test is in order to guarantee the feature added by https://github.com/apache/pulsar/pull/7105. + * 1. Start 3 consumers: + * - consumer1 will be closed and trigger a messages redeliver. + * - consumer2 will not ack any messages to make the new consumer joined late will be stuck due + * to the mechanism "recentlyJoinedConsumers". + * - consumer3 will always receive and ack messages. + * 2. Add consumer4 after consumer1 was close, and consumer4 will be stuck due to the mechanism + * "recentlyJoinedConsumers". + * 3. Verify: + * - (Main purpose) consumer3 can still receive messages util the cursor.readerPosition is larger than LAC. + * - no repeated Read-and-discard. + * - at last, all messages will be received. + */ + @Test(timeOut = 180 * 1000) // the test will be finished in 60s. + public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { + final int messagesSentPerTime = 100; + final Set totalReceivedMessages = new TreeSet<>(); + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "my-sub"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); + + // Send messages. + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); + for (int i = 0; i < messagesSentPerTime; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + + // 1. Start 3 consumers and make ack holes. + // - one consumer will be closed and trigger a messages redeliver. + // - one consumer will not ack any messages to make the new consumer joined late will be stuck due to the + // mechanism "recentlyJoinedConsumers". + // - one consumer will always receive and ack messages. + Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + List msgList1 = new ArrayList<>(); + List msgList2 = new ArrayList<>(); + List msgList3 = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg1 = consumer1.receive(1, TimeUnit.SECONDS); + if (msg1 != null) { + totalReceivedMessages.add(msg1.getValue()); + msgList1.add(msg1); + } + Message msg2 = consumer2.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + totalReceivedMessages.add(msg2.getValue()); + msgList2.add(msg2); + } + Message msg3 = consumer3.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + totalReceivedMessages.add(msg3.getValue()); + msgList3.add(msg3); + } + } + Consumer consumerWillBeClose = null; + Consumer consumerAlwaysAck = null; + Consumer consumerStuck = null; + if (!msgList1.isEmpty()) { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer1; + consumerWillBeClose = consumer2; + consumerStuck = consumer3; + } else if (!msgList2.isEmpty()){ + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer2; + consumerWillBeClose = consumer3; + consumerStuck = consumer1; + } else { + msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer3; + consumerWillBeClose = consumer1; + consumerStuck = consumer2; + } + + // 2. Add consumer4 after "consumerWillBeClose" was close, and consumer4 will be stuck due to the mechanism + // "recentlyJoinedConsumers". + Consumer consumer4 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(1000) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + consumerWillBeClose.close(); + + Thread.sleep(2000); + + for (int i = messagesSentPerTime; i < messagesSentPerTime * 2; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + + // Send messages again. + // Verify: "consumerAlwaysAck" can receive messages util the cursor.readerPosition is larger than LAC. + while (true) { + Message msg = consumerAlwaysAck.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + totalReceivedMessages.add(msg.getValue()); + consumerAlwaysAck.acknowledge(msg); + } + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor(subName); + log.info("cursor_readPosition {}, LAC {}", cursor.getReadPosition(), managedLedger.getLastConfirmedEntry()); + assertTrue(((PositionImpl) cursor.getReadPosition()) + .compareTo((PositionImpl) managedLedger.getLastConfirmedEntry()) > 0); + + // Make all consumers to start to read and acknowledge messages. + // Verify: no repeated Read-and-discard. + Thread.sleep(5 * 1000); + int maxReplayCount = messagesSentPerTime * 2; + log.info("Reply read count: {}", replyReadCounter.get()); + assertTrue(replyReadCounter.get() < maxReplayCount); + // Verify: at last, all messages will be received. + ReceivedMessages receivedMessages = ackAllMessages(consumerAlwaysAck, consumerStuck, consumer4); + totalReceivedMessages.addAll(receivedMessages.messagesReceived.stream().map(p -> p.getRight()).collect( + Collectors.toList())); + assertEquals(totalReceivedMessages.size(), messagesSentPerTime * 2); + + // cleanup. + consumer1.close(); + consumer2.close(); + consumer3.close(); + consumer4.close(); + producer.close(); + admin.topics().delete(topic, false); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java index bb8bab29ad9ef..7a12acd47edf9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java @@ -26,9 +26,11 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.google.common.collect.Lists; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -42,6 +44,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -50,6 +53,7 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; import org.mockito.AdditionalAnswers; import org.mockito.Mockito; @@ -372,6 +376,29 @@ public void testMultipleIOThreads() throws PulsarAdminException, PulsarClientExc assertTrue(consumer.isConnected()); } + @Test + public void testSameTopics() throws Exception { + final String topic1 = BrokerTestUtil.newUniqueName("public/default/tp"); + final String topic2 = "persistent://" + topic1; + admin.topics().createNonPartitionedTopic(topic2); + // Create consumer with two same topics. + try { + pulsarClient.newConsumer(Schema.INT32).topics(Arrays.asList(topic1, topic2)) + .subscriptionName("s1").subscribe(); + fail("Do not allow use two same topics."); + } catch (Exception e) { + if (e instanceof PulsarClientException && e.getCause() != null) { + e = (Exception) e.getCause(); + } + Throwable unwrapEx = FutureUtil.unwrapCompletionException(e); + assertTrue(unwrapEx instanceof IllegalArgumentException); + assertTrue(e.getMessage().contains( "Subscription topics include duplicate items" + + " or invalid names")); + } + // cleanup. + admin.topics().delete(topic2); + } + @Test(timeOut = 30000) public void testSubscriptionNotFound() throws PulsarAdminException, PulsarClientException { final var topic1 = newTopicName(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java index 6375f79bfbb6e..08b0777b640fc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java @@ -19,17 +19,35 @@ package org.apache.pulsar.client.api; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.AssertJUnit.assertTrue; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.PulsarChannelInitializer; import org.apache.pulsar.broker.service.ServerCnx; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.proto.CommandFlow; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; +import org.apache.pulsar.common.policies.data.SubscriptionStats; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -38,7 +56,7 @@ @Test(groups = "broker-api") @Slf4j -public class NonDurableSubscriptionTest extends ProducerConsumerBase { +public class NonDurableSubscriptionTest extends ProducerConsumerBase { private final AtomicInteger numFlow = new AtomicInteger(0); @@ -254,4 +272,400 @@ public void testFlowCountForMultiTopics() throws Exception { assertEquals(numFlow.get(), numPartitions); } + + private void trimLedgers(final String tpName) { + // Wait for topic loading. + org.awaitility.Awaitility.await().untilAsserted(() -> { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); + assertNotNull(persistentTopic); + }); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + CompletableFuture trimLedgersTask = new CompletableFuture<>(); + ml.trimConsumedLedgersInBackground(trimLedgersTask); + trimLedgersTask.join(); + } + + private void switchLedgerManually(final String tpName) throws Exception { + Method ledgerClosed = + ManagedLedgerImpl.class.getDeclaredMethod("ledgerClosed", new Class[]{LedgerHandle.class}); + Method createLedgerAfterClosed = + ManagedLedgerImpl.class.getDeclaredMethod("createLedgerAfterClosed", new Class[0]); + ledgerClosed.setAccessible(true); + createLedgerAfterClosed.setAccessible(true); + + // Wait for topic create. + org.awaitility.Awaitility.await().untilAsserted(() -> { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); + assertNotNull(persistentTopic); + }); + + // Switch ledger. + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + LedgerHandle currentLedger1 = WhiteboxImpl.getInternalState(ml, "currentLedger"); + ledgerClosed.invoke(ml, new Object[]{currentLedger1}); + createLedgerAfterClosed.invoke(ml, new Object[0]); + Awaitility.await().untilAsserted(() -> { + LedgerHandle currentLedger2 = WhiteboxImpl.getInternalState(ml, "currentLedger"); + assertNotEquals(currentLedger1.getId(), currentLedger2.getId()); + }); + } + + @Test + public void testHasMessageAvailableIfIncomingQueueNotEmpty() throws Exception { + final String nonDurableCursor = "non-durable-cursor"; + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + Reader reader = pulsarClient.newReader(Schema.STRING).topic(topicName).receiverQueueSize(1) + .subscriptionName(nonDurableCursor).startMessageId(MessageIdImpl.earliest).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + MessageIdImpl msgSent = (MessageIdImpl) producer.send("1"); + + // Trigger switch ledger. + // Trigger a trim ledgers task, and verify trim ledgers successful. + switchLedgerManually(topicName); + trimLedgers(topicName); + + // Since there is one message in the incoming queue, so the method "reader.hasMessageAvailable" should return + // true. + boolean hasMessageAvailable = reader.hasMessageAvailable(); + Message msgReceived = reader.readNext(2, TimeUnit.SECONDS); + if (msgReceived == null) { + assertFalse(hasMessageAvailable); + } else { + log.info("receive msg: {}", msgReceived.getValue()); + assertTrue(hasMessageAvailable); + assertEquals(msgReceived.getValue(), "1"); + } + + // cleanup. + reader.close(); + producer.close(); + admin.topics().delete(topicName); + } + + @Test + public void testInitReaderAtSpecifiedPosition() throws Exception { + String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, "s0", MessageId.earliest); + + // Trigger 5 ledgers. + ArrayList ledgers = new ArrayList<>(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + for (int i = 0; i < 5; i++) { + MessageIdImpl msgId = (MessageIdImpl) producer.send("1"); + ledgers.add(msgId.getLedgerId()); + admin.topics().unload(topicName); + } + producer.close(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + LedgerHandle currentLedger = WhiteboxImpl.getInternalState(ml, "currentLedger"); + log.info("currentLedger: {}", currentLedger.getId()); + + // Less than the first ledger, and entry id is "-1". + log.info("start test s1"); + String s1 = "s1"; + MessageIdImpl startMessageId1 = new MessageIdImpl(ledgers.get(0) - 1, -1, -1); + Reader reader1 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s1) + .receiverQueueSize(0).startMessageId(startMessageId1).create(); + ManagedLedgerInternalStats.CursorStats cursor1 = admin.topics().getInternalStats(topicName).cursors.get(s1); + log.info("cursor1 readPosition: {}, markDeletedPosition: {}", cursor1.readPosition, cursor1.markDeletePosition); + PositionImpl p1 = parseReadPosition(cursor1); + assertEquals(p1.getLedgerId(), ledgers.get(0)); + assertEquals(p1.getEntryId(), 0); + reader1.close(); + + // Less than the first ledger, and entry id is Long.MAX_VALUE. + log.info("start test s2"); + String s2 = "s2"; + MessageIdImpl startMessageId2 = new MessageIdImpl(ledgers.get(0) - 1, Long.MAX_VALUE, -1); + Reader reader2 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s2) + .receiverQueueSize(0).startMessageId(startMessageId2).create(); + ManagedLedgerInternalStats.CursorStats cursor2 = admin.topics().getInternalStats(topicName).cursors.get(s2); + log.info("cursor2 readPosition: {}, markDeletedPosition: {}", cursor2.readPosition, cursor2.markDeletePosition); + PositionImpl p2 = parseReadPosition(cursor2); + assertEquals(p2.getLedgerId(), ledgers.get(0)); + assertEquals(p2.getEntryId(), 0); + reader2.close(); + + // Larger than the latest ledger, and entry id is "-1". + log.info("start test s3"); + String s3 = "s3"; + MessageIdImpl startMessageId3 = new MessageIdImpl(currentLedger.getId() + 1, -1, -1); + Reader reader3 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s3) + .receiverQueueSize(0).startMessageId(startMessageId3).create(); + ManagedLedgerInternalStats.CursorStats cursor3 = admin.topics().getInternalStats(topicName).cursors.get(s3); + log.info("cursor3 readPosition: {}, markDeletedPosition: {}", cursor3.readPosition, cursor3.markDeletePosition); + PositionImpl p3 = parseReadPosition(cursor3); + assertEquals(p3.getLedgerId(), currentLedger.getId()); + assertEquals(p3.getEntryId(), 0); + reader3.close(); + + // Larger than the latest ledger, and entry id is Long.MAX_VALUE. + log.info("start test s4"); + String s4 = "s4"; + MessageIdImpl startMessageId4 = new MessageIdImpl(currentLedger.getId() + 1, Long.MAX_VALUE, -1); + Reader reader4 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s4) + .receiverQueueSize(0).startMessageId(startMessageId4).create(); + ManagedLedgerInternalStats.CursorStats cursor4 = admin.topics().getInternalStats(topicName).cursors.get(s4); + log.info("cursor4 readPosition: {}, markDeletedPosition: {}", cursor4.readPosition, cursor4.markDeletePosition); + PositionImpl p4 = parseReadPosition(cursor4); + assertEquals(p4.getLedgerId(), currentLedger.getId()); + assertEquals(p4.getEntryId(), 0); + reader4.close(); + + // Ledger id and entry id both are Long.MAX_VALUE. + log.info("start test s5"); + String s5 = "s5"; + MessageIdImpl startMessageId5 = new MessageIdImpl(currentLedger.getId() + 1, Long.MAX_VALUE, -1); + Reader reader5 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s5) + .receiverQueueSize(0).startMessageId(startMessageId5).create(); + ManagedLedgerInternalStats.CursorStats cursor5 = admin.topics().getInternalStats(topicName).cursors.get(s5); + log.info("cursor5 readPosition: {}, markDeletedPosition: {}", cursor5.readPosition, cursor5.markDeletePosition); + PositionImpl p5 = parseReadPosition(cursor5); + assertEquals(p5.getLedgerId(), currentLedger.getId()); + assertEquals(p5.getEntryId(), 0); + reader5.close(); + + // Ledger id equals LAC, and entry id is "-1". + log.info("start test s6"); + String s6 = "s6"; + MessageIdImpl startMessageId6 = new MessageIdImpl(ledgers.get(ledgers.size() - 1), -1, -1); + Reader reader6 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s6) + .receiverQueueSize(0).startMessageId(startMessageId6).create(); + ManagedLedgerInternalStats.CursorStats cursor6 = admin.topics().getInternalStats(topicName).cursors.get(s6); + log.info("cursor6 readPosition: {}, markDeletedPosition: {}", cursor6.readPosition, cursor6.markDeletePosition); + PositionImpl p6 = parseReadPosition(cursor6); + assertEquals(p6.getLedgerId(), ledgers.get(ledgers.size() - 1)); + assertEquals(p6.getEntryId(), 0); + reader6.close(); + + // Larger than the latest ledger, and entry id is Long.MAX_VALUE. + log.info("start test s7"); + String s7 = "s7"; + MessageIdImpl startMessageId7 = new MessageIdImpl(ledgers.get(ledgers.size() - 1), Long.MAX_VALUE, -1); + Reader reader7 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s7) + .receiverQueueSize(0).startMessageId(startMessageId7).create(); + ManagedLedgerInternalStats.CursorStats cursor7 = admin.topics().getInternalStats(topicName).cursors.get(s7); + log.info("cursor7 readPosition: {}, markDeletedPosition: {}", cursor7.readPosition, cursor7.markDeletePosition); + PositionImpl p7 = parseReadPosition(cursor7); + assertEquals(p7.getLedgerId(), currentLedger.getId()); + assertEquals(p7.getEntryId(), 0); + reader7.close(); + + // A middle ledger id, and entry id is "-1". + log.info("start test s8"); + String s8 = "s8"; + MessageIdImpl startMessageId8 = new MessageIdImpl(ledgers.get(2), 0, -1); + Reader reader8 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s8) + .receiverQueueSize(0).startMessageId(startMessageId8).create(); + ManagedLedgerInternalStats.CursorStats cursor8 = admin.topics().getInternalStats(topicName).cursors.get(s8); + log.info("cursor8 readPosition: {}, markDeletedPosition: {}", cursor8.readPosition, cursor8.markDeletePosition); + PositionImpl p8 = parseReadPosition(cursor8); + assertEquals(p8.getLedgerId(), ledgers.get(2)); + assertEquals(p8.getEntryId(), 0); + reader8.close(); + + // Larger than the latest ledger, and entry id is Long.MAX_VALUE. + log.info("start test s9"); + String s9 = "s9"; + MessageIdImpl startMessageId9 = new MessageIdImpl(ledgers.get(2), Long.MAX_VALUE, -1); + Reader reader9 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s9) + .receiverQueueSize(0).startMessageId(startMessageId9).create(); + ManagedLedgerInternalStats.CursorStats cursor9 = admin.topics().getInternalStats(topicName).cursors.get(s9); + log.info("cursor9 readPosition: {}, markDeletedPosition: {}", cursor9.readPosition, + cursor9.markDeletePosition); + PositionImpl p9 = parseReadPosition(cursor9); + assertEquals(p9.getLedgerId(), ledgers.get(3)); + assertEquals(p9.getEntryId(), 0); + reader9.close(); + + // Larger than the latest ledger, and entry id equals with the max entry id of this ledger. + log.info("start test s10"); + String s10 = "s10"; + MessageIdImpl startMessageId10 = new MessageIdImpl(ledgers.get(2), 0, -1); + Reader reader10 = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName(s10) + .receiverQueueSize(0).startMessageId(startMessageId10).create(); + ManagedLedgerInternalStats.CursorStats cursor10 = admin.topics().getInternalStats(topicName).cursors.get(s10); + log.info("cursor10 readPosition: {}, markDeletedPosition: {}", cursor10.readPosition, cursor10.markDeletePosition); + PositionImpl p10 = parseReadPosition(cursor10); + assertEquals(p10.getLedgerId(), ledgers.get(2)); + assertEquals(p10.getEntryId(), 0); + reader10.close(); + + // cleanup + admin.topics().delete(topicName, false); + } + + private PositionImpl parseReadPosition(ManagedLedgerInternalStats.CursorStats cursorStats) { + String[] ledgerIdAndEntryId = cursorStats.readPosition.split(":"); + return PositionImpl.get(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + } + + @Test + public void testReaderInitAtDeletedPosition() throws Exception { + String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topicName); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + producer.send("1"); + producer.send("2"); + producer.send("3"); + MessageIdImpl msgIdInDeletedLedger4 = (MessageIdImpl) producer.send("4"); + MessageIdImpl msgIdInDeletedLedger5 = (MessageIdImpl) producer.send("5"); + + // Trigger a trim ledgers task, and verify trim ledgers successful. + admin.topics().unload(topicName); + trimLedgers(topicName); + List ledgers = admin.topics().getInternalStats(topicName).ledgers; + assertEquals(ledgers.size(), 1); + assertNotEquals(ledgers.get(0).ledgerId, msgIdInDeletedLedger5.getLedgerId()); + + // Start a reader at a deleted ledger. + MessageIdImpl startMessageId = + new MessageIdImpl(msgIdInDeletedLedger4.getLedgerId(), msgIdInDeletedLedger4.getEntryId(), -1); + Reader reader = pulsarClient.newReader(Schema.STRING).topic(topicName).subscriptionName("s1") + .startMessageId(startMessageId).create(); + Message msg1 = reader.readNext(2, TimeUnit.SECONDS); + Assert.assertNull(msg1); + + // Verify backlog and markDeletePosition is correct. + Awaitility.await().untilAsserted(() -> { + SubscriptionStats subscriptionStats = admin.topics() + .getStats(topicName, true, true, true).getSubscriptions().get("s1"); + log.info("backlog size: {}", subscriptionStats.getMsgBacklog()); + assertEquals(subscriptionStats.getMsgBacklog(), 0); + ManagedLedgerInternalStats.CursorStats cursorStats = + admin.topics().getInternalStats(topicName).cursors.get("s1"); + String[] ledgerIdAndEntryId = cursorStats.markDeletePosition.split(":"); + PositionImpl actMarkDeletedPos = + PositionImpl.get(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + PositionImpl expectedMarkDeletedPos = + PositionImpl.get(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); + log.info("Expected mark deleted position: {}", expectedMarkDeletedPos); + log.info("Actual mark deleted position: {}", cursorStats.markDeletePosition); + assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); + }); + + // cleanup. + reader.close(); + producer.close(); + admin.topics().delete(topicName, false); + } + + @Test + public void testTrimLedgerIfNoDurableCursor() throws Exception { + final String nonDurableCursor = "non-durable-cursor"; + final String durableCursor = "durable-cursor"; + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topicName); + Reader reader = pulsarClient.newReader(Schema.STRING).topic(topicName).receiverQueueSize(1) + .subscriptionName(nonDurableCursor).startMessageId(MessageIdImpl.earliest).create(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).receiverQueueSize(1) + .subscriptionName(durableCursor).subscribe(); + consumer.close(); + + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + producer.send("1"); + producer.send("2"); + producer.send("3"); + producer.send("4"); + MessageIdImpl msgIdInDeletedLedger5 = (MessageIdImpl) producer.send("5"); + + Message msg1 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg1.getValue(), "1"); + Message msg2 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg2.getValue(), "2"); + Message msg3 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg3.getValue(), "3"); + + // Unsubscribe durable cursor. + // Trigger a trim ledgers task, and verify trim ledgers successful. + admin.topics().unload(topicName); + Thread.sleep(3 * 1000); + admin.topics().deleteSubscription(topicName, durableCursor); + // Trim ledgers after release durable cursor. + trimLedgers(topicName); + List ledgers = admin.topics().getInternalStats(topicName).ledgers; + assertEquals(ledgers.size(), 1); + assertNotEquals(ledgers.get(0).ledgerId, msgIdInDeletedLedger5.getLedgerId()); + + // Verify backlog and markDeletePosition is correct. + Awaitility.await().untilAsserted(() -> { + SubscriptionStats subscriptionStats = admin.topics().getStats(topicName, true, true, true) + .getSubscriptions().get(nonDurableCursor); + log.info("backlog size: {}", subscriptionStats.getMsgBacklog()); + assertEquals(subscriptionStats.getMsgBacklog(), 0); + ManagedLedgerInternalStats.CursorStats cursorStats = + admin.topics().getInternalStats(topicName).cursors.get(nonDurableCursor); + String[] ledgerIdAndEntryId = cursorStats.markDeletePosition.split(":"); + PositionImpl actMarkDeletedPos = + PositionImpl.get(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + PositionImpl expectedMarkDeletedPos = + PositionImpl.get(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); + log.info("Expected mark deleted position: {}", expectedMarkDeletedPos); + log.info("Actual mark deleted position: {}", cursorStats.markDeletePosition); + Assert.assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); + }); + + // Clear the incoming queue of the reader for next test. + while (true) { + Message msg = reader.readNext(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + log.info("clear msg: {}", msg.getValue()); + } + + // The following tests are designed to verify the api "getNumberOfEntries" and "consumedEntries" still work + // after changes.See the code-description added with the PR https://github.com/apache/pulsar/pull/10667. + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(nonDurableCursor); + + // Verify "getNumberOfEntries" if there is no entries to consume. + assertEquals(0, cursor.getNumberOfEntries()); + assertEquals(0, ml.getNumberOfEntries()); + + // Verify "getNumberOfEntries" if there is 1 entry to consume. + producer.send("6"); + producer.send("7"); + Awaitility.await().untilAsserted(() -> { + assertEquals(2, ml.getNumberOfEntries()); + // Since there is one message has been pulled into the incoming queue of reader. There is only one messages + // waiting to cursor read. + assertEquals(1, cursor.getNumberOfEntries()); + }); + + // Verify "consumedEntries" is correct. + ManagedLedgerInternalStats.CursorStats cursorStats = + admin.topics().getInternalStats(topicName).cursors.get(nonDurableCursor); + // "messagesConsumedCounter" should be 0 after unload the topic. + // Note: "topic_internal_stat.cursor.messagesConsumedCounter" means how many messages were acked on this + // cursor. The similar one "topic_stats.lastConsumedTimestamp" means the last time of sending messages to + // the consumer. + assertEquals(0, cursorStats.messagesConsumedCounter); + Message msg6 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg6.getValue(), "6"); + Message msg7 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg7.getValue(), "7"); + Awaitility.await().untilAsserted(() -> { + // "messagesConsumedCounter" should be 2 after consumed 2 message. + ManagedLedgerInternalStats.CursorStats cStat = + admin.topics().getInternalStats(topicName).cursors.get(nonDurableCursor); + assertEquals(2, cStat.messagesConsumedCounter); + }); + + // cleanup. + reader.close(); + producer.close(); + admin.topics().delete(topicName, false); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index 7cd9da7574dbb..7a6189702dd8c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -20,7 +20,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; -import java.lang.reflect.Field; import java.util.List; import java.util.Map; import java.util.Optional; @@ -29,14 +28,12 @@ import java.util.concurrent.atomic.AtomicInteger; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; -import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.service.TopicPolicyListener; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; -import org.apache.pulsar.compaction.CompactionServiceFactory; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.testng.annotations.AfterClass; @@ -107,48 +104,4 @@ public void testNoOrphanTopicAfterCreateTimeout() throws Exception { admin.topics().delete(tpName, false); pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); } - - @Test - public void testNoOrphanTopicIfInitFailed() throws Exception { - String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); - admin.topics().createNonPartitionedTopic(tpName); - - // Load topic. - Consumer consumer = pulsarClient.newConsumer() - .topic(tpName) - .subscriptionName("my-sub") - .subscribe(); - - // Make the method `PersitentTopic.initialize` fail. - Field fieldCompactionServiceFactory = PulsarService.class.getDeclaredField("compactionServiceFactory"); - fieldCompactionServiceFactory.setAccessible(true); - CompactionServiceFactory compactionServiceFactory = - (CompactionServiceFactory) fieldCompactionServiceFactory.get(pulsar); - fieldCompactionServiceFactory.set(pulsar, null); - admin.topics().unload(tpName); - - // Wait for failed to create topic for several times. - Thread.sleep(5 * 1000); - - // Remove the injected error, the topic will be created successful. - fieldCompactionServiceFactory.set(pulsar, compactionServiceFactory); - // We do not know the next time of consumer reconnection, so wait for 2 minutes to avoid flaky. It will be - // very fast in normal. - Awaitility.await().ignoreExceptions().atMost(120, TimeUnit.SECONDS).untilAsserted(() -> { - CompletableFuture> future = pulsar.getBrokerService().getTopic(tpName, false); - assertTrue(future.isDone()); - Optional optional = future.get(); - assertTrue(optional.isPresent()); - }); - - // Assert only one PersistentTopic was not closed. - TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService(); - Map>> listeners = - WhiteboxImpl.getInternalState(topicPoliciesService, "listeners"); - assertEquals(listeners.get(TopicName.get(tpName)).size(), 1); - - // cleanup. - consumer.close(); - admin.topics().delete(tpName, false); - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java index f58c1fa26afc7..ef070250ca1aa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java @@ -21,9 +21,14 @@ import com.google.common.collect.Sets; import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; import java.util.Random; import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -69,4 +74,65 @@ protected String newTopicName() { return "my-property/my-ns/topic-" + Long.toHexString(random.nextLong()); } + protected ReceivedMessages receiveAndAckMessages( + BiFunction ackPredicate, + Consumer...consumers) throws Exception { + ReceivedMessages receivedMessages = new ReceivedMessages(); + while (true) { + int receivedMsgCount = 0; + for (int i = 0; i < consumers.length; i++) { + Consumer consumer = consumers[i]; + while (true) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + if (msg != null) { + receivedMsgCount++; + T v = msg.getValue(); + MessageId messageId = msg.getMessageId(); + receivedMessages.messagesReceived.add(Pair.of(msg.getMessageId(), v)); + if (ackPredicate.apply(messageId, v)) { + consumer.acknowledge(msg); + receivedMessages.messagesAcked.add(Pair.of(msg.getMessageId(), v)); + } + } else { + break; + } + } + } + // Because of the possibility of consumers getting stuck with each other, only jump out of the loop if all + // consumers could not receive messages. + if (receivedMsgCount == 0) { + break; + } + } + return receivedMessages; + } + + protected ReceivedMessages ackAllMessages(Consumer...consumers) throws Exception { + return receiveAndAckMessages((msgId, msgV) -> true, consumers); + } + + protected static class ReceivedMessages { + + List> messagesReceived = new ArrayList<>(); + + List> messagesAcked = new ArrayList<>(); + + public boolean hasReceivedMessage(T v) { + for (Pair pair : messagesReceived) { + if (pair.getRight().equals(v)) { + return true; + } + } + return false; + } + + public boolean hasAckedMessage(T v) { + for (Pair pair : messagesAcked) { + if (pair.getRight().equals(v)) { + return true; + } + } + return false; + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java new file mode 100644 index 0000000000000..ab4e063ae3d83 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java @@ -0,0 +1,108 @@ +/* + * 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 org.apache.pulsar.client.api; + +import com.carrotsearch.hppc.ObjectSet; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.common.naming.TopicName; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class SimpleProducerConsumerMLInitializeDelayTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setTopicLoadTimeoutSeconds(60 * 5); + } + + @Test(timeOut = 30 * 1000) + public void testConsumerListMatchesConsumerSet() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "sub"; + final int clientOperationTimeout = 3; + final int loadMLDelayMillis = clientOperationTimeout * 3 * 1000; + final int clientMaxBackoffSeconds = clientOperationTimeout * 2; + admin.topics().createNonPartitionedTopic(topicName); + // Create a client with a low operation timeout. + PulsarClient client = PulsarClient.builder() + .serviceUrl(lookupUrl.toString()) + .operationTimeout(clientOperationTimeout, TimeUnit.SECONDS) + .maxBackoffInterval(clientMaxBackoffSeconds, TimeUnit.SECONDS) + .build(); + Consumer consumer = client.newConsumer() + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + // Inject a delay for the initialization of ML, to make the consumer to register twice. + // Consumer register twice: the first will be timeout, and try again. + AtomicInteger delayTimes = new AtomicInteger(); + mockZooKeeper.delay(loadMLDelayMillis, (op, s) -> { + if (op.toString().equals("GET") && s.contains(TopicName.get(topicName).getPersistenceNamingEncoding())) { + return delayTimes.incrementAndGet() == 1; + } + return false; + }); + admin.topics().unload(topicName); + // Verify: at last, "dispatcher.consumers.size" equals "dispatcher.consumerList.size". + Awaitility.await().atMost(Duration.ofSeconds(loadMLDelayMillis * 3)) + .ignoreExceptions().untilAsserted(() -> { + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subName).getDispatcher(); + ObjectSet consumerSet = WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + List consumerList = WhiteboxImpl.getInternalState(dispatcher, "consumerList"); + log.info("consumerSet_size: {}, consumerList_size: {}", consumerSet.size(), consumerList.size()); + Assert.assertEquals(consumerList.size(), 1); + Assert.assertEquals(consumerSet.size(), 1); + }); + + // Verify: the topic can be deleted. + consumer.close(); + admin.topics().delete(topicName); + // cleanup. + client.close(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 4536bda907b6a..7552b84a1c553 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -51,6 +51,7 @@ import java.time.Instant; import java.time.ZoneId; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -69,6 +70,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -92,18 +94,21 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.ClientBuilderImpl; -import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.ConsumerBase; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.PartitionedProducerImpl; +import org.apache.pulsar.client.impl.ProducerBase; +import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.TopicMessageImpl; import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; import org.apache.pulsar.client.impl.schema.writer.AvroWriter; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.EncryptionContext; import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey; import org.apache.pulsar.common.api.proto.MessageMetadata; @@ -3906,11 +3911,11 @@ public void testReleaseSemaphoreOnFailMessages() throws Exception { .topic("persistent://my-property/my-ns/my-topic2"); @Cleanup - Producer producer = producerBuilder.create(); + ProducerImpl producer = (ProducerImpl)producerBuilder.create(); List> futures = new ArrayList<>(); // Asynchronously produce messages - byte[] message = new byte[ClientCnx.getMaxMessageSize() + 1]; + byte[] message = new byte[producer.getConnectionHandler().getMaxMessageSize() + 1]; for (int i = 0; i < maxPendingMessages + 10; i++) { Future future = producer.sendAsync(message); try { @@ -4692,4 +4697,143 @@ public void flush(ChannelHandlerContext ctx) throws Exception { consumer.close(); admin.topics().delete(topic, false); } + + @DataProvider(name = "enableBatchSend") + public Object[][] enableBatchSend() { + return new Object[][]{ + {true}, + {false} + }; + } + + @Test(dataProvider = "enableBatchSend") + public void testPublishWithCreateMessageManually(boolean enableBatchSend) throws Exception { + final int messageCount = 10; + final List messageArrayBeforeSend = Collections.synchronizedList(new ArrayList<>()); + final List messageArrayOnSendAcknowledgement = Collections.synchronizedList(new ArrayList<>()); + // Create an interceptor to verify the ref count of Message.payload is as expected. + AtomicBoolean payloadWasReleasedWhenIntercept = new AtomicBoolean(false); + ProducerInterceptor interceptor = new ProducerInterceptor(){ + + @Override + public void close() { + + } + @Override + public Message beforeSend(Producer producer, Message message) { + MessageImpl msgImpl = (MessageImpl) message; + log.info("payload.refCnf before send: {}", msgImpl.getDataBuffer().refCnt()); + if (msgImpl.getDataBuffer().refCnt() < 1) { + payloadWasReleasedWhenIntercept.set(true); + } + messageArrayBeforeSend.add(msgImpl); + return message; + } + + @Override + public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, + Throwable exception) { + MessageImpl msgImpl = (MessageImpl) message; + log.info("payload.refCnf on send acknowledgement: {}", msgImpl.getDataBuffer().refCnt()); + if (msgImpl.getDataBuffer().refCnt() < 1) { + payloadWasReleasedWhenIntercept.set(true); + } + messageArrayOnSendAcknowledgement.add(msgImpl); + } + }; + + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp"); + admin.topics().createNonPartitionedTopic(topic); + ProducerBase producerBase = (ProducerBase) pulsarClient.newProducer().topic(topic).intercept(interceptor) + .enableBatching(enableBatchSend).create(); + + // Publish message. + // Note: "ProducerBase.sendAsync" is not equals to "Producer.sendAsync". + final MessageImpl[] messageArraySent = new MessageImpl[messageCount]; + final ByteBuf[] payloads = new ByteBuf[messageCount]; + List> sendFutureList = new ArrayList<>(); + List releaseFutureList = new ArrayList<>(); + for (int i = 0; i < messageCount; i++) { + // Create message payload, refCnf = 1 now. + ByteBuf payload = PulsarByteBufAllocator.DEFAULT.heapBuffer(1); + payloads[i] = payload; + log.info("payload_{}.refCnf 1st: {}", i, payload.refCnt()); + payload.writeByte(i); + // refCnf = 2 now. + payload.retain(); + log.info("payload_{}.refCnf 2nd: {}", i, payload.refCnt()); + MessageMetadata messageMetadata = new MessageMetadata(); + messageMetadata.setUncompressedSize(1); + MessageImpl message1 = MessageImpl.create(topic, null, messageMetadata, payload, Optional.empty(), + null, Schema.BYTES, 0, true, 0); + messageArraySent[i] = message1; + // Release ByteBuf the first time, refCnf = 1 now. + CompletableFuture future = producerBase.sendAsync(message1); + sendFutureList.add(future); + final int indexForLog = i; + future.whenComplete((v, ex) -> { + message1.release(); + log.info("payload_{}.refCnf 3rd after_complete_refCnf: {}, ex: {}", indexForLog, payload.refCnt(), + ex == null ? "null" : ex.getMessage()); + }); + } + sendFutureList.get(messageCount - 1).join(); + + // Left 2 seconds to wait the code in the finally-block, which is using to avoid this test to be flaky. + Thread.sleep(1000 * 2); + + // Verify: payload's refCnf. + for (int i = 0; i < messageCount; i++) { + log.info("payload_{}.refCnf 4th: {}", i, payloads[i].refCnt()); + assertEquals(payloads[i].refCnt(), 1); + } + + // Verify: the messages has not been released when calling interceptor. + assertFalse(payloadWasReleasedWhenIntercept.get()); + + // Verify: the order of send complete event. + MessageIdImpl messageIdPreviousOne = null; + for (int i = 0; i < messageCount; i++) { + MessageIdImpl messageId = (MessageIdImpl) sendFutureList.get(i).get(); + if (messageIdPreviousOne != null) { + assertTrue(compareMessageIds(messageIdPreviousOne, messageId) > 0); + } + messageIdPreviousOne = messageId; + } + + // Verify: the order of interceptor events. + for (int i = 0; i < messageCount; i++) { + assertTrue(messageArraySent[i] == messageArrayBeforeSend.get(i)); + assertTrue(messageArraySent[i] == messageArrayOnSendAcknowledgement.get(i)); + } + + // cleanup. + for (int i = 0; i < messageCount; i++) { + payloads[i].release(); + } + producerBase.close(); + admin.topics().delete(topic, false); + } + + private int compareMessageIds(MessageIdImpl messageId1, MessageIdImpl messageId2) { + if (messageId2.getLedgerId() < messageId1.getLedgerId()) { + return -1; + } + if (messageId2.getLedgerId() > messageId1.getLedgerId()) { + return 1; + } + if (messageId2.getEntryId() < messageId1.getEntryId()) { + return -1; + } + if (messageId2.getEntryId() > messageId1.getEntryId()) { + return 1; + } + if (messageId2 instanceof BatchMessageIdImpl && messageId1 instanceof BatchMessageIdImpl) { + BatchMessageIdImpl batchMessageId1 = (BatchMessageIdImpl) messageId1; + BatchMessageIdImpl batchMessageId2 = (BatchMessageIdImpl) messageId2; + return batchMessageId2.getBatchIndex() - batchMessageId1.getBatchIndex(); + } else { + return 0; + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicNameForInfiniteHttpCallGetSubscriptionsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicNameForInfiniteHttpCallGetSubscriptionsTest.java new file mode 100644 index 0000000000000..96f57a59bda13 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicNameForInfiniteHttpCallGetSubscriptionsTest.java @@ -0,0 +1,146 @@ +/* + * 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 org.apache.pulsar.client.api; + +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicType; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class TopicNameForInfiniteHttpCallGetSubscriptionsTest extends ProducerConsumerBase { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + conf.setDefaultNumPartitions(1); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testInfiniteHttpCallGetOrCreateSubscriptions() throws Exception { + final String randomStr = UUID.randomUUID().toString().replaceAll("-", ""); + final String partitionedTopicName = "persistent://my-property/my-ns/tp1_" + randomStr; + final String topic_p0 = partitionedTopicName + TopicName.PARTITIONED_TOPIC_SUFFIX + "0"; + final String subscriptionName = "sub1"; + final String topicDLQ = topic_p0 + "-" + subscriptionName + "-DLQ"; + + admin.topics().createPartitionedTopic(partitionedTopicName, 2); + + // Do test. + ProducerAndConsumerEntry pcEntry = triggerDLQCreated(topic_p0, topicDLQ, subscriptionName); + admin.topics().getSubscriptions(topicDLQ); + admin.topics().createSubscription(topicDLQ, "s1", MessageId.earliest); + + // cleanup. + pcEntry.consumer.close(); + pcEntry.producer.close(); + admin.topics().deletePartitionedTopic(partitionedTopicName); + } + + @Test + public void testInfiniteHttpCallGetOrCreateSubscriptions2() throws Exception { + final String randomStr = UUID.randomUUID().toString().replaceAll("-", ""); + final String topicName = "persistent://my-property/my-ns/tp1_" + randomStr + "-partition-0-abc"; + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .create(); + + // Do test. + admin.topics().getSubscriptions(topicName); + admin.topics().createSubscription(topicName, "s1", MessageId.earliest); + + // cleanup. + producer.close(); + } + + @Test + public void testInfiniteHttpCallGetOrCreateSubscriptions3() throws Exception { + final String randomStr = UUID.randomUUID().toString().replaceAll("-", ""); + final String topicName = "persistent://my-property/my-ns/tp1_" + randomStr + "-partition-0"; + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .create(); + + // Do test. + admin.topics().getSubscriptions(topicName); + admin.topics().createSubscription(topicName, "s1", MessageId.earliest); + + // cleanup. + producer.close(); + } + + @AllArgsConstructor + private static class ProducerAndConsumerEntry { + private Producer producer; + private Consumer consumer; + } + + private ProducerAndConsumerEntry triggerDLQCreated(String topicName, String DLQName, String subscriptionName) throws Exception { + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Shared) + .enableRetry(true) + .deadLetterPolicy(DeadLetterPolicy.builder().deadLetterTopic(DLQName).maxRedeliverCount(2).build()) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .create(); + // send messages. + for (int i = 0; i < 5; i++) { + producer.newMessage() + .value("value-" + i) + .sendAsync(); + } + producer.flush(); + // trigger the DLQ created. + for (int i = 0; i < 20; i++) { + Message msg = consumer.receive(5, TimeUnit.SECONDS); + if (msg != null) { + consumer.reconsumeLater(msg, 1, TimeUnit.SECONDS); + } else { + break; + } + } + + return new ProducerAndConsumerEntry(producer, consumer); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java index 4ec8107030600..a06085d3d4626 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java @@ -20,6 +20,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -29,11 +30,14 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.client.api.BatcherBuilder; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.api.proto.CommandCloseProducer; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -186,4 +190,44 @@ public void testSendTimerCheckForBatchContainer(BatcherBuilder batcherBuilder) t future.thenAccept(msgId -> log.info("msg-1 done: {} (msgId: {})", System.nanoTime(), msgId)); future.get(); } + + + @Test + public void testRetentionPolicyByProducingMessages() throws Exception { + // Setup: configure the entries per ledger and retention polices. + final int maxEntriesPerLedger = 10, messagesCount = 10; + final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + pulsar.getConfiguration().setManagedLedgerMaxEntriesPerLedger(maxEntriesPerLedger); + pulsar.getConfiguration().setManagedLedgerMinLedgerRolloverTimeMinutes(0); + pulsar.getConfiguration().setDefaultRetentionTimeInMinutes(0); + pulsar.getConfiguration().setDefaultRetentionSizeInMB(0); + + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topicName) + .sendTimeout(1, TimeUnit.SECONDS) + .enableBatching(false) + .create(); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer().topic(topicName) + .subscriptionName("my-sub") + .subscribe(); + // Act: prepare a full ledger data and ack them. + for (int i = 0; i < messagesCount; i++) { + producer.newMessage().sendAsync(); + } + for (int i = 0; i < messagesCount; i++) { + Message message = consumer.receive(); + assertNotNull(message); + consumer.acknowledge(message); + } + // Verify: a new empty ledger will be created after the current ledger is fulled. + // And the previous consumed ledgers will be deleted + Awaitility.await().untilAsserted(() -> { + admin.topics().trimTopic(topicName); + PersistentTopicInternalStats internalStats = admin.topics().getInternalStatsAsync(topicName).get(); + assertEquals(internalStats.currentLedgerEntries, 0); + assertEquals(internalStats.ledgers.size(), 1); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java index d776fdb0ed915..55a67ae644d36 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java @@ -21,6 +21,7 @@ import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import io.netty.buffer.ByteBufAllocator; import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; @@ -33,7 +34,6 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SizeUnit; -import org.mockito.MockedStatic; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -69,10 +69,12 @@ public void testProducerInvalidMessageMemoryRelease() throws Exception { .create(); this.stopBroker(); try { - try (MockedStatic mockedStatic = Mockito.mockStatic(ClientCnx.class)) { - mockedStatic.when(ClientCnx::getMaxMessageSize).thenReturn(8); - producer.send("memory-test".getBytes(StandardCharsets.UTF_8)); - } + ConnectionHandler connectionHandler = Mockito.spy(producer.getConnectionHandler()); + Field field = producer.getClass().getDeclaredField("connectionHandler"); + field.setAccessible(true); + field.set(producer, connectionHandler); + when(connectionHandler.getMaxMessageSize()).thenReturn(8); + producer.send("memory-test".getBytes(StandardCharsets.UTF_8)); throw new IllegalStateException("can not reach here"); } catch (PulsarClientException.InvalidMessageException ex) { PulsarClientImpl clientImpl = (PulsarClientImpl) this.pulsarClient; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSemaphoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSemaphoreTest.java index 2f8cb655401d9..42f431e0b9b53 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSemaphoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSemaphoreTest.java @@ -22,6 +22,7 @@ import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import io.netty.buffer.ByteBufAllocator; import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; @@ -33,7 +34,6 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.util.FutureUtil; -import org.mockito.MockedStatic; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -72,13 +72,14 @@ public void testProducerSemaphoreInvalidMessage() throws Exception { .maxPendingMessages(pendingQueueSize) .enableBatching(true) .create(); - this.stopBroker(); try { - try (MockedStatic mockedStatic = Mockito.mockStatic(ClientCnx.class)) { - mockedStatic.when(ClientCnx::getMaxMessageSize).thenReturn(2); - producer.send("semaphore-test".getBytes(StandardCharsets.UTF_8)); - } + ConnectionHandler connectionHandler = Mockito.spy(producer.getConnectionHandler()); + Field field = producer.getClass().getDeclaredField("connectionHandler"); + field.setAccessible(true); + field.set(producer, connectionHandler); + when(connectionHandler.getMaxMessageSize()).thenReturn(2); + producer.send("semaphore-test".getBytes(StandardCharsets.UTF_8)); throw new IllegalStateException("can not reach here"); } catch (PulsarClientException.InvalidMessageException ex) { Assert.assertEquals(producer.getSemaphore().get().availablePermits(), pendingQueueSize); @@ -86,10 +87,7 @@ public void testProducerSemaphoreInvalidMessage() throws Exception { producer.conf.setBatchingEnabled(false); try { - try (MockedStatic mockedStatic = Mockito.mockStatic(ClientCnx.class)) { - mockedStatic.when(ClientCnx::getMaxMessageSize).thenReturn(2); - producer.send("semaphore-test".getBytes(StandardCharsets.UTF_8)); - } + producer.send("semaphore-test".getBytes(StandardCharsets.UTF_8)); throw new IllegalStateException("can not reach here"); } catch (PulsarClientException.InvalidMessageException ex) { Assert.assertEquals(producer.getSemaphore().get().availablePermits(), pendingQueueSize); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java index 95d8926c9ff80..d3fcc36a54653 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -36,15 +37,18 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.api.RawReader; import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; import org.awaitility.Awaitility; import org.testng.Assert; @@ -496,4 +500,23 @@ public void testReadCancellationOnClose() throws Exception { } } } + + @Test + public void testAutoCreateTopic() throws ExecutionException, InterruptedException, PulsarAdminException { + String topic = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader"); + + RawReader reader = RawReader.create(pulsarClient, topic, subscription).get(); + TopicStats stats = admin.topics().getStats(topic); + Assert.assertNotNull(stats); + reader.closeAsync().join(); + + String topic2 = "persistent://my-property/my-ns/" + BrokerTestUtil.newUniqueName("reader"); + try { + reader = RawReader.create(pulsarClient, topic2, subscription, false).get(); + Assert.fail(); + } catch (Exception e) { + Assert.assertTrue(e.getCause() instanceof PulsarClientException.TopicDoesNotExistException); + } + reader.closeAsync().join(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index 85ff87593cffa..0e46445af69a1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -44,6 +44,7 @@ import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; @@ -64,8 +65,9 @@ import org.apache.pulsar.schema.Schemas; import org.awaitility.Awaitility; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -74,7 +76,7 @@ public class ReaderTest extends MockedPulsarServiceBaseTest { private static final String subscription = "reader-sub"; - @BeforeMethod + @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { super.internalSetup(); @@ -86,7 +88,7 @@ protected void setup() throws Exception { admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); } - @AfterMethod(alwaysRun = true) + @AfterClass(alwaysRun = true) @Override protected void cleanup() throws Exception { super.internalCleanup(); @@ -145,21 +147,41 @@ public void testReadMessageWithBatching() throws Exception { testReadMessages(topic, true); } - @Test - public void testReadMessageWithBatchingWithMessageInclusive() throws Exception { + @DataProvider + public static Object[][] seekBeforeHasMessageAvailable() { + return new Object[][] { { true }, { false } }; + } + + @Test(timeOut = 20000, dataProvider = "seekBeforeHasMessageAvailable") + public void testReadMessageWithBatchingWithMessageInclusive(boolean seekBeforeHasMessageAvailable) + throws Exception { String topic = "persistent://my-property/my-ns/my-reader-topic-with-batching-inclusive"; Set keys = publishMessages(topic, 10, true); Reader reader = pulsarClient.newReader().topic(topic).startMessageId(MessageId.latest) .startMessageIdInclusive().readerName(subscription).create(); - while (reader.hasMessageAvailable()) { - Assert.assertTrue(keys.remove(reader.readNext().getKey())); + if (seekBeforeHasMessageAvailable) { + reader.seek(0L); // it should seek to the earliest } + + assertTrue(reader.hasMessageAvailable()); + final Message msg = reader.readNext(); + assertTrue(keys.remove(msg.getKey())); // start from latest with start message inclusive should only read the last message in batch assertEquals(keys.size(), 9); - Assert.assertFalse(keys.contains("key9")); - Assert.assertFalse(reader.hasMessageAvailable()); + + final MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + if (seekBeforeHasMessageAvailable) { + assertEquals(msgId.getBatchIndex(), 0); + assertFalse(keys.contains("key0")); + assertTrue(reader.hasMessageAvailable()); + } else { + assertEquals(msgId.getBatchIndex(), 9); + assertFalse(reader.hasMessageAvailable()); + assertFalse(keys.contains("key9")); + assertFalse(reader.hasMessageAvailable()); + } } private void testReadMessages(String topic, boolean enableBatch) throws Exception { @@ -257,7 +279,7 @@ public void testReadFromPartition() throws Exception { @Test public void testReaderWithTimeLong() throws Exception { String ns = "my-property/my-ns"; - String topic = "persistent://" + ns + "/testReadFromPartition"; + String topic = "persistent://" + ns + "/testReaderWithTimeLong"; RetentionPolicies retention = new RetentionPolicies(-1, -1); admin.namespaces().setRetention(ns, retention); @@ -761,4 +783,72 @@ public void testReaderReconnectedFromNextEntry() throws Exception { producer.close(); admin.topics().delete(topic, false); } + + @DataProvider + public static Object[][] initializeLastMessageIdInBroker() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "initializeLastMessageIdInBroker") + public void testHasMessageAvailableAfterSeek(boolean initializeLastMessageIdInBroker) throws Exception { + final String topic = "persistent://my-property/my-ns/test-has-message-available-after-seek"; + @Cleanup Reader reader = pulsarClient.newReader(Schema.STRING).topic(topic).receiverQueueSize(1) + .startMessageId(MessageId.earliest).create(); + + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("msg"); + + if (initializeLastMessageIdInBroker) { + assertTrue(reader.hasMessageAvailable()); + } // else: lastMessageIdInBroker is earliest + + reader.seek(MessageId.latest); + // lastMessageIdInBroker is the last message ID, while startMessageId is still earliest + assertFalse(reader.hasMessageAvailable()); + + producer.send("msg"); + assertTrue(reader.hasMessageAvailable()); + } + + @Test(dataProvider = "initializeLastMessageIdInBroker") + public void testHasMessageAvailableAfterSeekTimestamp(boolean initializeLastMessageIdInBroker) throws Exception { + final String topic = "persistent://my-property/my-ns/test-has-message-available-after-seek-timestamp"; + + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + final long timestampBeforeSend = System.currentTimeMillis(); + final MessageId sentMsgId = producer.send("msg"); + + final List messageIds = new ArrayList<>(); + messageIds.add(MessageId.earliest); + messageIds.add(sentMsgId); + messageIds.add(MessageId.latest); + + for (MessageId messageId : messageIds) { + @Cleanup Reader reader = pulsarClient.newReader(Schema.STRING).topic(topic).receiverQueueSize(1) + .startMessageId(messageId).create(); + if (initializeLastMessageIdInBroker) { + if (messageId == MessageId.earliest) { + assertTrue(reader.hasMessageAvailable()); + } else { + assertFalse(reader.hasMessageAvailable()); + } + } // else: lastMessageIdInBroker is earliest + reader.seek(System.currentTimeMillis()); + assertFalse(reader.hasMessageAvailable()); + } + + for (MessageId messageId : messageIds) { + @Cleanup Reader reader = pulsarClient.newReader(Schema.STRING).topic(topic).receiverQueueSize(1) + .startMessageId(messageId).create(); + if (initializeLastMessageIdInBroker) { + if (messageId == MessageId.earliest) { + assertTrue(reader.hasMessageAvailable()); + } else { + assertFalse(reader.hasMessageAvailable()); + } + } // else: lastMessageIdInBroker is earliest + reader.seek(timestampBeforeSend); + assertTrue(reader.hasMessageAvailable()); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index 71700ef83a443..b651cf05da2dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -19,12 +19,15 @@ package org.apache.pulsar.compaction; import static org.apache.pulsar.client.impl.RawReaderTest.extractKey; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; + import io.netty.buffer.ByteBuf; + import java.util.ArrayList; import java.util.Enumeration; import java.util.HashMap; @@ -32,28 +35,23 @@ import java.util.Map; import java.util.Optional; import java.util.Random; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; + import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; -import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; @@ -61,8 +59,8 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.RawMessageImpl; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.testng.Assert; @@ -79,6 +77,7 @@ public class CompactorTest extends MockedPulsarServiceBaseTest { protected Compactor compactor; + @BeforeMethod @Override public void setup() throws Exception { @@ -114,17 +113,16 @@ protected Compactor getCompactor() { return compactor; } - private List compactAndVerify(String topic, Map expected, boolean checkMetrics) - throws Exception { + private List compactAndVerify(String topic, Map expected, boolean checkMetrics) throws Exception { long compactedLedgerId = compact(topic); LedgerHandle ledger = bk.openLedger(compactedLedgerId, - Compactor.COMPACTED_TOPIC_LEDGER_DIGEST_TYPE, - Compactor.COMPACTED_TOPIC_LEDGER_PASSWORD); + Compactor.COMPACTED_TOPIC_LEDGER_DIGEST_TYPE, + Compactor.COMPACTED_TOPIC_LEDGER_PASSWORD); Assert.assertEquals(ledger.getLastAddConfirmed() + 1, // 0..lac - expected.size(), - "Should have as many entries as there is keys"); + expected.size(), + "Should have as many entries as there is keys"); List keys = new ArrayList<>(); Enumeration entries = ledger.readEntries(0, ledger.getLastAddConfirmed()); @@ -138,7 +136,7 @@ private List compactAndVerify(String topic, Map expected byte[] bytes = new byte[payload.readableBytes()]; payload.readBytes(bytes); Assert.assertEquals(bytes, expected.remove(key), - "Compacted version should match expected version"); + "Compacted version should match expected version"); m.close(); } if (checkMetrics) { @@ -162,18 +160,17 @@ public void testCompaction() throws Exception { final int numMessages = 1000; final int maxKeys = 10; - @Cleanup Producer producer = pulsarClient.newProducer().topic(topic) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); Map expected = new HashMap<>(); Random r = new Random(0); for (int j = 0; j < numMessages; j++) { int keyIndex = r.nextInt(maxKeys); - String key = "key" + keyIndex; + String key = "key"+keyIndex; byte[] data = ("my-message-" + key + "-" + j).getBytes(); producer.newMessage() .key(key) @@ -237,11 +234,10 @@ public void testAllCompactedOut() throws Exception { public void testCompactAddCompact() throws Exception { String topic = "persistent://my-property/use/my-ns/my-topic1"; - @Cleanup Producer producer = pulsarClient.newProducer().topic(topic) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); Map expected = new HashMap<>(); @@ -275,11 +271,10 @@ public void testCompactAddCompact() throws Exception { public void testCompactedInOrder() throws Exception { String topic = "persistent://my-property/use/my-ns/my-topic1"; - @Cleanup Producer producer = pulsarClient.newProducer().topic(topic) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); producer.newMessage() .key("c") @@ -325,48 +320,6 @@ public void testPhaseOneLoopTimeConfiguration() { Assert.assertEquals(compactor.getPhaseOneLoopReadTimeoutInSeconds(), 60); } - @Test - public void testCompactedWithConcurrentSend() throws Exception { - String topic = "persistent://my-property/use/my-ns/testCompactedWithConcurrentSend"; - - @Cleanup - Producer producer = pulsarClient.newProducer().topic(topic) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); - - var future = CompletableFuture.runAsync(() -> { - for (int i = 0; i < 100; i++) { - try { - producer.newMessage().key(String.valueOf(i)).value(String.valueOf(i).getBytes()).send(); - } catch (PulsarClientException e) { - throw new RuntimeException(e); - } - } - }); - - PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); - PulsarTopicCompactionService topicCompactionService = - (PulsarTopicCompactionService) persistentTopic.getTopicCompactionService(); - - Awaitility.await().untilAsserted(() -> { - long compactedLedgerId = compact(topic); - Thread.sleep(300); - Optional compactedTopicContext = topicCompactionService.getCompactedTopic() - .getCompactedTopicContext(); - Assert.assertTrue(compactedTopicContext.isPresent()); - Assert.assertEquals(compactedTopicContext.get().ledger.getId(), compactedLedgerId); - }); - - Position lastCompactedPosition = topicCompactionService.getLastCompactedPosition().get(); - Entry lastCompactedEntry = topicCompactionService.readLastCompactedEntry().get(); - - Assert.assertTrue(PositionImpl.get(lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) - .compareTo(lastCompactedEntry.getLedgerId(), lastCompactedEntry.getEntryId()) >= 0); - - future.join(); - } - public ByteBuf extractPayload(RawMessage m) throws Exception { ByteBuf payloadAndMetadata = m.getHeadersAndPayload(); Commands.skipChecksumIfPresent(payloadAndMetadata); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java index 5901d7c177ea7..5f2006edeafdf 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java @@ -1961,7 +1961,8 @@ private class GetOffloadThreshold extends CliCommand { @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - print(getAdmin().namespaces().getOffloadThreshold(namespace)); + print("offloadThresholdInBytes: " + getAdmin().namespaces().getOffloadThreshold(namespace)); + print("offloadThresholdInSeconds: " + getAdmin().namespaces().getOffloadThresholdInSeconds(namespace)); } } @@ -1979,11 +1980,18 @@ private class SetOffloadThreshold extends CliCommand { required = true) private String thresholdStr = "-1"; + @Parameter(names = {"--time", "-t"}, + description = "Maximum number of seconds stored on the pulsar cluster for a topic" + + " before the broker will start offloading to longterm storage (eg: 10m, 5h, 3d, 2w).") + private String thresholdInSeconds = "-1"; + @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); long threshold = validateSizeString(thresholdStr); + long timeInSeconds = RelativeTimeUtil.parseRelativeTimeInSeconds(thresholdInSeconds); getAdmin().namespaces().setOffloadThreshold(namespace, threshold); + getAdmin().namespaces().setOffloadThresholdInSeconds(namespace, timeInSeconds); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AbstractBatchMessageContainer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AbstractBatchMessageContainer.java index 8c17d8fcb253c..3ba7866350a0c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AbstractBatchMessageContainer.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AbstractBatchMessageContainer.java @@ -25,6 +25,7 @@ import org.apache.pulsar.common.api.proto.CompressionType; import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; +import org.apache.pulsar.common.protocol.Commands; /** * Batch message container framework. @@ -59,14 +60,18 @@ public abstract class AbstractBatchMessageContainer implements BatchMessageConta public boolean haveEnoughSpace(MessageImpl msg) { int messageSize = msg.getDataBuffer().readableBytes(); return ( - (maxBytesInBatch <= 0 && (messageSize + currentBatchSizeBytes) <= ClientCnx.getMaxMessageSize()) + (maxBytesInBatch <= 0 && (messageSize + currentBatchSizeBytes) <= getMaxMessageSize()) || (maxBytesInBatch > 0 && (messageSize + currentBatchSizeBytes) <= maxBytesInBatch) ) && (maxNumMessagesInBatch <= 0 || numMessagesInBatch < maxNumMessagesInBatch); } + protected int getMaxMessageSize() { + return producer != null && producer.getConnectionHandler() != null + ? producer.getConnectionHandler().getMaxMessageSize() : Commands.DEFAULT_MAX_MESSAGE_SIZE; + } protected boolean isBatchFull() { return (maxBytesInBatch > 0 && currentBatchSizeBytes >= maxBytesInBatch) - || (maxBytesInBatch <= 0 && currentBatchSizeBytes >= ClientCnx.getMaxMessageSize()) + || (maxBytesInBatch <= 0 && currentBatchSizeBytes >= getMaxMessageSize()) || (maxNumMessagesInBatch > 0 && numMessagesInBatch >= maxNumMessagesInBatch); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java index bf8c1f9de8201..fc5c3a3c6798b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java @@ -101,7 +101,7 @@ public boolean add(MessageImpl msg, SendCallback callback) { lowestSequenceId = Commands.initBatchMessageMetadata(messageMetadata, msg.getMessageBuilder()); this.firstCallback = callback; batchedMessageMetadataAndPayload = allocator.buffer( - Math.min(maxBatchSize, ClientCnx.getMaxMessageSize())); + Math.min(maxBatchSize, getMaxMessageSize())); updateAndReserveBatchAllocatedSize(batchedMessageMetadataAndPayload.capacity()); if (msg.getMessageBuilder().hasTxnidMostBits() && currentTxnidMostBits == -1) { currentTxnidMostBits = msg.getMessageBuilder().getTxnidMostBits(); @@ -272,12 +272,12 @@ public OpSendMsg createOpSendMsg() throws IOException { op.setBatchSizeByte(encryptedPayload.readableBytes()); // handle mgs size check as non-batched in `ProducerImpl.isMessageSizeExceeded` - if (op.getMessageHeaderAndPayloadSize() > ClientCnx.getMaxMessageSize()) { + if (op.getMessageHeaderAndPayloadSize() > getMaxMessageSize()) { producer.semaphoreRelease(1); producer.client.getMemoryLimitController().releaseMemory( messages.get(0).getUncompressedSize() + batchAllocatedSizeBytes); discard(new PulsarClientException.InvalidMessageException( - "Message size is bigger than " + ClientCnx.getMaxMessageSize() + " bytes")); + "Message size is bigger than " + getMaxMessageSize() + " bytes")); return null; } lowestSequenceId = -1L; @@ -285,13 +285,13 @@ public OpSendMsg createOpSendMsg() throws IOException { } ByteBuf encryptedPayload = producer.encryptMessage(messageMetadata, getCompressedBatchMetadataAndPayload()); updateAndReserveBatchAllocatedSize(encryptedPayload.capacity()); - if (encryptedPayload.readableBytes() > ClientCnx.getMaxMessageSize()) { + if (encryptedPayload.readableBytes() > getMaxMessageSize()) { producer.semaphoreRelease(messages.size()); messages.forEach(msg -> producer.client.getMemoryLimitController() .releaseMemory(msg.getUncompressedSize())); producer.client.getMemoryLimitController().releaseMemory(batchAllocatedSizeBytes); discard(new PulsarClientException.InvalidMessageException( - "Message size is bigger than " + ClientCnx.getMaxMessageSize() + " bytes")); + "Message size is bigger than " + getMaxMessageSize() + " bytes")); return null; } messageMetadata.setNumMessagesInBatch(numMessagesInBatch); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index e1b374acbbb54..8694dad7a2d84 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -30,6 +30,7 @@ import io.netty.channel.EventLoopGroup; import io.netty.channel.unix.Errors.NativeIoException; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.handler.ssl.SslHandshakeCompletionEvent; import io.netty.util.concurrent.Promise; import java.net.InetSocketAddress; import java.net.SocketAddress; @@ -166,8 +167,7 @@ public class ClientCnx extends PulsarHandler { private volatile int numberOfRejectRequests = 0; @Getter - private static int maxMessageSize = Commands.DEFAULT_MAX_MESSAGE_SIZE; - + private int maxMessageSize = Commands.DEFAULT_MAX_MESSAGE_SIZE; private final int maxNumberOfRejectedRequestPerConnection; private final int rejectedRequestResetTimeSec = 60; protected final int protocolVersion; @@ -1313,6 +1313,18 @@ public void close() { } } + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof SslHandshakeCompletionEvent) { + SslHandshakeCompletionEvent sslHandshakeCompletionEvent = (SslHandshakeCompletionEvent) evt; + if (sslHandshakeCompletionEvent.cause() != null) { + log.warn("{} Got ssl handshake exception {}", ctx.channel(), + sslHandshakeCompletionEvent); + } + } + ctx.fireUserEventTriggered(evt); + } + protected void closeWithException(Throwable e) { if (ctx != null) { connectionFuture.completeExceptionally(e); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index fc7c89c3ce693..77a85c255a8f8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -24,8 +24,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.Getter; +import lombok.Setter; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.HandlerState.State; +import org.apache.pulsar.common.protocol.Commands; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +37,10 @@ public class ConnectionHandler { AtomicReferenceFieldUpdater.newUpdater(ConnectionHandler.class, ClientCnx.class, "clientCnx"); @SuppressWarnings("unused") private volatile ClientCnx clientCnx = null; + @Getter + @Setter + // Since the `clientCnx` variable will be set to null at some times, it is necessary to save this value here. + private volatile int maxMessageSize = Commands.DEFAULT_MAX_MESSAGE_SIZE; protected final HandlerState state; protected final Backoff backoff; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index fa5a8bf3c5aa4..1e6dc401f93b7 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -166,7 +166,9 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private volatile MessageIdAdv startMessageId; private volatile MessageIdAdv seekMessageId; - private final AtomicBoolean duringSeek; + @VisibleForTesting + final AtomicReference seekStatus; + private volatile CompletableFuture seekFuture; private final MessageIdAdv initialStartMessageId; @@ -215,6 +217,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final AtomicReference clientCnxUsedForConsumerRegistration = new AtomicReference<>(); private final List previousExceptions = new CopyOnWriteArrayList(); + private volatile boolean hasSoughtByTimestamp = false; static ConsumerImpl newConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, @@ -303,7 +306,7 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat stats = ConsumerStatsDisabled.INSTANCE; } - duringSeek = new AtomicBoolean(false); + seekStatus = new AtomicReference<>(SeekStatus.NOT_STARTED); // Create msgCrypto if not created already if (conf.getCryptoKeyReader() != null) { @@ -771,6 +774,7 @@ public void negativeAcknowledge(Message message) { @Override public CompletableFuture connectionOpened(final ClientCnx cnx) { previousExceptions.clear(); + getConnectionHandler().setMaxMessageSize(cnx.getMaxMessageSize()); final State state = getState(); if (state == State.Closing || state == State.Closed) { @@ -778,7 +782,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { closeConsumerTasks(); deregisterFromClientCnx(); client.cleanupConsumer(this); - clearReceiverQueue(); + clearReceiverQueue(false); return CompletableFuture.completedFuture(null); } @@ -786,7 +790,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { topic, subscription, cnx.ctx().channel(), consumerId); long requestId = client.newRequestId(); - if (duringSeek.get()) { + if (seekStatus.get() != SeekStatus.NOT_STARTED) { acknowledgmentsGroupingTracker.flushAndClean(); } @@ -797,7 +801,8 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { int currentSize; synchronized (this) { currentSize = incomingMessages.size(); - startMessageId = clearReceiverQueue(); + setClientCnx(cnx); + clearReceiverQueue(true); if (possibleSendToDeadLetterTopicMessages != null) { possibleSendToDeadLetterTopicMessages.clear(); } @@ -835,7 +840,6 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // synchronized this, because redeliverUnAckMessage eliminate the epoch inconsistency between them final CompletableFuture future = new CompletableFuture<>(); synchronized (this) { - setClientCnx(cnx); ByteBuf request = Commands.newSubscribe(topic, subscription, consumerId, requestId, getSubType(), priorityLevel, consumerName, isDurable, startMessageIdData, metadata, readCompacted, conf.isReplicateSubscriptionState(), @@ -940,15 +944,24 @@ protected void consumerIsReconnectedToBroker(ClientCnx cnx, int currentQueueSize * Clear the internal receiver queue and returns the message id of what was the 1st message in the queue that was * not seen by the application. */ - private MessageIdAdv clearReceiverQueue() { + private void clearReceiverQueue(boolean updateStartMessageId) { List> currentMessageQueue = new ArrayList<>(incomingMessages.size()); incomingMessages.drainTo(currentMessageQueue); resetIncomingMessageSize(); - if (duringSeek.compareAndSet(true, false)) { - return seekMessageId; + CompletableFuture seekFuture = this.seekFuture; + MessageIdAdv seekMessageId = this.seekMessageId; + + if (seekStatus.get() != SeekStatus.NOT_STARTED) { + if (updateStartMessageId) { + startMessageId = seekMessageId; + } + if (seekStatus.compareAndSet(SeekStatus.COMPLETED, SeekStatus.NOT_STARTED)) { + internalPinnedExecutor.execute(() -> seekFuture.complete(null)); + } + return; } else if (subscriptionMode == SubscriptionMode.Durable) { - return startMessageId; + return; } if (!currentMessageQueue.isEmpty()) { @@ -965,15 +978,14 @@ private MessageIdAdv clearReceiverQueue() { } // release messages if they are pooled messages currentMessageQueue.forEach(Message::release); - return previousMessage; - } else if (!lastDequeuedMessageId.equals(MessageId.earliest)) { + if (updateStartMessageId) { + startMessageId = previousMessage; + } + } else if (updateStartMessageId && !lastDequeuedMessageId.equals(MessageId.earliest)) { // If the queue was empty we need to restart from the message just after the last one that has been dequeued // in the past - return new BatchMessageIdImpl((MessageIdImpl) lastDequeuedMessageId); - } else { - // No message was received or dequeued by this consumer. Next message would still be the startMessageId - return startMessageId; - } + startMessageId = new BatchMessageIdImpl((MessageIdImpl) lastDequeuedMessageId); + } // else: No message was received or dequeued by this consumer. Next message would still be the startMessageId } /** @@ -1882,7 +1894,7 @@ private ByteBuf uncompressPayloadIfNeeded(MessageIdData messageId, MessageMetada CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(compressionType); int uncompressedSize = msgMetadata.getUncompressedSize(); int payloadSize = payload.readableBytes(); - if (checkMaxMessageSize && payloadSize > ClientCnx.getMaxMessageSize()) { + if (checkMaxMessageSize && payloadSize > getConnectionHandler().getMaxMessageSize()) { // payload size is itself corrupted since it cannot be bigger than the MaxMessageSize log.error("[{}][{}] Got corrupted payload message size {} at {}", topic, subscription, payloadSize, messageId); @@ -2232,100 +2244,128 @@ public CompletableFuture seekAsync(Function function) { new PulsarClientException("Only support seek by messageId or timestamp")); } - private Optional> seekAsyncCheckState(String seekBy) { - if (getState() == State.Closing || getState() == State.Closed) { - return Optional.of(FutureUtil - .failedFuture(new PulsarClientException.AlreadyClosedException( - String.format("The consumer %s was already closed when seeking the subscription %s of the" - + " topic %s to %s", consumerName, subscription, topicName.toString(), seekBy)))); - } - - if (!isConnected()) { - return Optional.of(FutureUtil.failedFuture(new PulsarClientException( - String.format("The client is not connected to the broker when seeking the subscription %s of the " - + "topic %s to %s", subscription, topicName.toString(), seekBy)))); - } - - return Optional.empty(); - } - - private CompletableFuture seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, String seekBy) { - final CompletableFuture seekFuture = new CompletableFuture<>(); - ClientCnx cnx = cnx(); + private CompletableFuture seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, + Long seekTimestamp, String seekBy) { + AtomicLong opTimeoutMs = new AtomicLong(client.getConfiguration().getOperationTimeoutMs()); + Backoff backoff = new BackoffBuilder() + .setInitialTime(100, TimeUnit.MILLISECONDS) + .setMax(opTimeoutMs.get() * 2, TimeUnit.MILLISECONDS) + .setMandatoryStop(0, TimeUnit.MILLISECONDS) + .create(); - if (!duringSeek.compareAndSet(false, true)) { + if (!seekStatus.compareAndSet(SeekStatus.NOT_STARTED, SeekStatus.IN_PROGRESS)) { final String message = String.format( "[%s][%s] attempting to seek operation that is already in progress (seek by %s)", topic, subscription, seekBy); log.warn("[{}][{}] Attempting to seek operation that is already in progress, cancelling {}", topic, subscription, seekBy); - seekFuture.completeExceptionally(new IllegalStateException(message)); - return seekFuture; + return FutureUtil.failedFuture(new IllegalStateException(message)); } + seekFuture = new CompletableFuture<>(); + seekAsyncInternal(requestId, seek, seekId, seekTimestamp, seekBy, backoff, opTimeoutMs); + return seekFuture; + } - MessageIdAdv originSeekMessageId = seekMessageId; - seekMessageId = (MessageIdAdv) seekId; - log.info("[{}][{}] Seeking subscription to {}", topic, subscription, seekBy); + private void seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, Long seekTimestamp, String seekBy, + final Backoff backoff, final AtomicLong remainingTime) { + ClientCnx cnx = cnx(); + if (isConnected() && cnx != null) { + MessageIdAdv originSeekMessageId = seekMessageId; + seekMessageId = (MessageIdAdv) seekId; + log.info("[{}][{}] Seeking subscription to {}", topic, subscription, seekBy); - cnx.sendRequestWithId(seek, requestId).thenRun(() -> { - log.info("[{}][{}] Successfully reset subscription to {}", topic, subscription, seekBy); - acknowledgmentsGroupingTracker.flushAndClean(); + final boolean originalHasSoughtByTimestamp = hasSoughtByTimestamp; + hasSoughtByTimestamp = (seekTimestamp != null); + cnx.sendRequestWithId(seek, requestId).thenRun(() -> { + log.info("[{}][{}] Successfully reset subscription to {}", topic, subscription, seekBy); + acknowledgmentsGroupingTracker.flushAndClean(); - lastDequeuedMessageId = MessageId.earliest; + lastDequeuedMessageId = MessageId.earliest; + + clearIncomingMessages(); + CompletableFuture future = null; + synchronized (this) { + if (!hasParentConsumer && cnx() == null) { + // It's during reconnection, complete the seek future after connection is established + seekStatus.set(SeekStatus.COMPLETED); + } else { + future = seekFuture; + startMessageId = seekMessageId; + seekStatus.set(SeekStatus.NOT_STARTED); + } + } + if (future != null) { + future.complete(null); + } + }).exceptionally(e -> { + seekMessageId = originSeekMessageId; + hasSoughtByTimestamp = originalHasSoughtByTimestamp; + log.error("[{}][{}] Failed to reset subscription: {}", topic, subscription, e.getCause().getMessage()); + + failSeek( + PulsarClientException.wrap(e.getCause(), + String.format("Failed to seek the subscription %s of the topic %s to %s", + subscription, topicName.toString(), seekBy))); + return null; + }); + } else { + long nextDelay = Math.min(backoff.next(), remainingTime.get()); + if (nextDelay <= 0) { + failSeek( + new PulsarClientException.TimeoutException( + String.format("The subscription %s of the topic %s could not seek " + + "withing configured timeout", subscription, topicName.toString()))); + return; + } - clearIncomingMessages(); - seekFuture.complete(null); - }).exceptionally(e -> { - // re-set duringSeek and seekMessageId if seek failed - seekMessageId = originSeekMessageId; - duringSeek.set(false); - log.error("[{}][{}] Failed to reset subscription: {}", topic, subscription, e.getCause().getMessage()); + ((ScheduledExecutorService) client.getScheduledExecutorProvider().getExecutor()).schedule(() -> { + log.warn("[{}] [{}] Could not get connection while seek -- Will try again in {} ms", + topic, getHandlerName(), nextDelay); + remainingTime.addAndGet(-nextDelay); + seekAsyncInternal(requestId, seek, seekId, seekTimestamp, seekBy, backoff, remainingTime); + }, nextDelay, TimeUnit.MILLISECONDS); + } + } - seekFuture.completeExceptionally( - PulsarClientException.wrap(e.getCause(), - String.format("Failed to seek the subscription %s of the topic %s to %s", - subscription, topicName.toString(), seekBy))); - return null; - }); - return seekFuture; + private void failSeek(Throwable throwable) { + CompletableFuture seekFuture = this.seekFuture; + if (seekStatus.compareAndSet(SeekStatus.IN_PROGRESS, SeekStatus.NOT_STARTED)) { + seekFuture.completeExceptionally(throwable); + } } @Override public CompletableFuture seekAsync(long timestamp) { String seekBy = String.format("the timestamp %d", timestamp); - return seekAsyncCheckState(seekBy).orElseGet(() -> { - long requestId = client.newRequestId(); - return seekAsyncInternal(requestId, Commands.newSeek(consumerId, requestId, timestamp), - MessageId.earliest, seekBy); - }); + long requestId = client.newRequestId(); + return seekAsyncInternal(requestId, Commands.newSeek(consumerId, requestId, timestamp), + MessageId.earliest, timestamp, seekBy); } @Override public CompletableFuture seekAsync(MessageId messageId) { String seekBy = String.format("the message %s", messageId.toString()); - return seekAsyncCheckState(seekBy).orElseGet(() -> { - long requestId = client.newRequestId(); - final MessageIdAdv msgId = (MessageIdAdv) messageId; - final MessageIdAdv firstChunkMsgId = msgId.getFirstChunkMessageId(); - final ByteBuf seek; - if (msgId.getFirstChunkMessageId() != null) { - seek = Commands.newSeek(consumerId, requestId, firstChunkMsgId.getLedgerId(), - firstChunkMsgId.getEntryId(), new long[0]); + long requestId = client.newRequestId(); + final MessageIdAdv msgId = (MessageIdAdv) messageId; + final MessageIdAdv firstChunkMsgId = msgId.getFirstChunkMessageId(); + final ByteBuf seek; + if (msgId.getFirstChunkMessageId() != null) { + seek = Commands.newSeek(consumerId, requestId, firstChunkMsgId.getLedgerId(), + firstChunkMsgId.getEntryId(), new long[0]); + } else { + final long[] ackSetArr; + if (MessageIdAdvUtils.isBatch(msgId)) { + final BitSetRecyclable ackSet = BitSetRecyclable.create(); + ackSet.set(0, msgId.getBatchSize()); + ackSet.clear(0, Math.max(msgId.getBatchIndex(), 0)); + ackSetArr = ackSet.toLongArray(); + ackSet.recycle(); } else { - final long[] ackSetArr; - if (MessageIdAdvUtils.isBatch(msgId)) { - final BitSetRecyclable ackSet = BitSetRecyclable.create(); - ackSet.set(0, msgId.getBatchSize()); - ackSet.clear(0, Math.max(msgId.getBatchIndex(), 0)); - ackSetArr = ackSet.toLongArray(); - ackSet.recycle(); - } else { - ackSetArr = new long[0]; - } - seek = Commands.newSeek(consumerId, requestId, msgId.getLedgerId(), msgId.getEntryId(), ackSetArr); + ackSetArr = new long[0]; } - return seekAsyncInternal(requestId, seek, messageId, seekBy); - }); + seek = Commands.newSeek(consumerId, requestId, msgId.getLedgerId(), msgId.getEntryId(), ackSetArr); + } + return seekAsyncInternal(requestId, seek, messageId, null, seekBy); } public boolean hasMessageAvailable() throws PulsarClientException { @@ -2339,15 +2379,21 @@ public boolean hasMessageAvailable() throws PulsarClientException { public CompletableFuture hasMessageAvailableAsync() { final CompletableFuture booleanFuture = new CompletableFuture<>(); + if (incomingMessages != null && !incomingMessages.isEmpty()) { + return CompletableFuture.completedFuture(true); + } + // we haven't read yet. use startMessageId for comparison if (lastDequeuedMessageId == MessageId.earliest) { + // If the last seek is called with timestamp, startMessageId cannot represent the position to start, so we + // have to get the mark-delete position from the GetLastMessageId response to compare as well. // if we are starting from latest, we should seek to the actual last message first. // allow the last one to be read when read head inclusively. - if (MessageId.latest.equals(startMessageId)) { - + final boolean hasSoughtByTimestamp = this.hasSoughtByTimestamp; + if (MessageId.latest.equals(startMessageId) || hasSoughtByTimestamp) { CompletableFuture future = internalGetLastMessageIdAsync(); // if the consumer is configured to read inclusive then we need to seek to the last message - if (resetIncludeHead) { + if (resetIncludeHead && !hasSoughtByTimestamp) { future = future.thenCompose((lastMessageIdResponse) -> seekAsync(lastMessageIdResponse.lastMessageId) .thenApply((ignore) -> lastMessageIdResponse)); @@ -2947,4 +2993,10 @@ boolean isAckReceiptEnabled() { private static final Logger log = LoggerFactory.getLogger(ConsumerImpl.class); + @VisibleForTesting + enum SeekStatus { + NOT_STARTED, + IN_PROGRESS, + COMPLETED + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java index a0d1446ba3d55..f66bb64202115 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java @@ -40,6 +40,13 @@ static boolean equals(MessageIdAdv lhs, Object o) { && lhs.getBatchIndex() == rhs.getBatchIndex(); } + /** + * Acknowledge batch message. + * + * @param msgId the message id + * @param individual whether to acknowledge the batch message individually + * @return true if the batch message is fully acknowledged + */ static boolean acknowledge(MessageIdAdv msgId, boolean individual) { if (!isBatch(msgId)) { return true; @@ -51,12 +58,14 @@ static boolean acknowledge(MessageIdAdv msgId, boolean individual) { return false; } int batchIndex = msgId.getBatchIndex(); - if (individual) { - ackSet.clear(batchIndex); - } else { - ackSet.clear(0, batchIndex + 1); + synchronized (ackSet) { + if (individual) { + ackSet.clear(batchIndex); + } else { + ackSet.clear(0, batchIndex + 1); + } + return ackSet.isEmpty(); } - return ackSet.isEmpty(); } static boolean isBatch(MessageIdAdv msgId) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 0272d3d3baf34..f2bce59a1e68e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -165,7 +165,8 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { return; } - checkArgument(topicNamesValid(conf.getTopicNames()), "Topics is invalid."); + checkArgument(topicNamesValid(conf.getTopicNames()), "Subscription topics include duplicate items" + + " or invalid names."); List> futures = conf.getTopicNames().stream() .map(t -> subscribeAsync(t, createTopicIfDoesNotExist)) @@ -202,21 +203,21 @@ private static boolean topicNamesValid(Collection topics) { checkState(topics != null && topics.size() >= 1, "topics should contain more than 1 topic"); - Optional result = topics.stream() - .filter(topic -> !TopicName.isValid(topic)) - .findFirst(); + Set topicNames = new HashSet<>(); - if (result.isPresent()) { - log.warn("Received invalid topic name: {}", result.get()); - return false; + for (String topic : topics) { + if (!TopicName.isValid(topic)) { + log.warn("Received invalid topic name: {}", topic); + return false; + } + topicNames.add(TopicName.get(topic)); } // check topic names are unique - HashSet set = new HashSet<>(topics); - if (set.size() == topics.size()) { + if (topicNames.size() == topics.size()) { return true; } else { - log.warn("Topic names not unique. unique/all : {}/{}", set.size(), topics.size()); + log.warn("Topic names not unique. unique/all : {}/{}", topicNames.size(), topics.size()); return false; } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 0cf776aea5942..c0ee13b346a0b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -324,8 +324,15 @@ private CompletableFuture doIndividualBatchAckAsync(MessageIdAdv msgId) { MessageIdAdvUtils.discardBatch(msgId), __ -> { final BitSet ackSet = msgId.getAckSet(); final ConcurrentBitSetRecyclable value; - if (ackSet != null && !ackSet.isEmpty()) { - value = ConcurrentBitSetRecyclable.create(ackSet); + if (ackSet != null) { + synchronized (ackSet) { + if (!ackSet.isEmpty()) { + value = ConcurrentBitSetRecyclable.create(ackSet); + } else { + value = ConcurrentBitSetRecyclable.create(); + value.set(0, msgId.getBatchSize()); + } + } } else { value = ConcurrentBitSetRecyclable.create(); value.set(0, msgId.getBatchSize()); @@ -374,8 +381,11 @@ private CompletableFuture doImmediateBatchIndexAck(MessageIdAdv msgId, int .ConnectException("Consumer connect fail! consumer state:" + consumer.getState())); } BitSetRecyclable bitSet; - if (msgId.getAckSet() != null) { - bitSet = BitSetRecyclable.valueOf(msgId.getAckSet().toLongArray()); + BitSet ackSetFromMsgId = msgId.getAckSet(); + if (ackSetFromMsgId != null) { + synchronized (ackSetFromMsgId) { + bitSet = BitSetRecyclable.valueOf(ackSetFromMsgId.toLongArray()); + } } else { bitSet = BitSetRecyclable.create(); bitSet.set(0, batchSize); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 3fc3bbf3cf783..a41de6d10512d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -179,9 +179,6 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration this.userProvidedProducerName = StringUtils.isNotBlank(producerName); this.partitionIndex = partitionIndex; this.pendingMessages = createPendingMessagesQueue(); - this.chunkMaxMessageSize = conf.getChunkMaxMessageSize() > 0 - ? Math.min(conf.getChunkMaxMessageSize(), ClientCnx.getMaxMessageSize()) - : ClientCnx.getMaxMessageSize(); if (conf.getMaxPendingMessages() > 0) { this.semaphore = Optional.of(new Semaphore(conf.getMaxPendingMessages(), true)); } else { @@ -274,10 +271,16 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration .setMandatoryStop(Math.max(100, conf.getSendTimeoutMs() - 100), TimeUnit.MILLISECONDS) .create(), this); - + setChunkMaxMessageSize(); grabCnx(); } + private void setChunkMaxMessageSize() { + this.chunkMaxMessageSize = conf.getChunkMaxMessageSize() > 0 + ? Math.min(conf.getChunkMaxMessageSize(), getMaxMessageSize()) + : getMaxMessageSize(); + } + protected void semaphoreRelease(final int releaseCountRequest) { if (semaphore.isPresent()) { if (!errorState) { @@ -333,73 +336,78 @@ CompletableFuture internalSendAsync(Message message) { if (interceptors != null) { interceptorMessage.getProperties(); } - sendAsync(interceptorMessage, new SendCallback() { - SendCallback nextCallback = null; - MessageImpl nextMsg = null; - long createdAt = System.nanoTime(); - @Override - public CompletableFuture getFuture() { - return future; - } + int msgSize = interceptorMessage.getDataBuffer().readableBytes(); + sendAsync(interceptorMessage, new DefaultSendMessageCallback(future, interceptorMessage, msgSize)); + return future; + } - @Override - public SendCallback getNextSendCallback() { - return nextCallback; - } + private class DefaultSendMessageCallback implements SendCallback { - @Override - public MessageImpl getNextMessage() { - return nextMsg; - } + CompletableFuture sendFuture; + MessageImpl currentMsg; + long createdAt = System.nanoTime(); + SendCallback nextCallback = null; + MessageImpl nextMsg = null; - @Override - public void sendComplete(Exception e) { - try { - if (e != null) { - stats.incrementSendFailed(); - onSendAcknowledgement(interceptorMessage, null, e); - future.completeExceptionally(e); - } else { - onSendAcknowledgement(interceptorMessage, interceptorMessage.getMessageId(), null); - future.complete(interceptorMessage.getMessageId()); - stats.incrementNumAcksReceived(System.nanoTime() - createdAt); - } - } finally { - interceptorMessage.getDataBuffer().release(); - } + DefaultSendMessageCallback(CompletableFuture sendFuture, MessageImpl currentMsg, int msgSize) { + this.sendFuture = sendFuture; + this.currentMsg = currentMsg; + } - while (nextCallback != null) { - SendCallback sendCallback = nextCallback; - MessageImpl msg = nextMsg; - // Retain the buffer used by interceptors callback to get message. Buffer will release after - // complete interceptors. - try { - msg.getDataBuffer().retain(); - if (e != null) { - stats.incrementSendFailed(); - onSendAcknowledgement(msg, null, e); - sendCallback.getFuture().completeExceptionally(e); - } else { - onSendAcknowledgement(msg, msg.getMessageId(), null); - sendCallback.getFuture().complete(msg.getMessageId()); - stats.incrementNumAcksReceived(System.nanoTime() - createdAt); - } - nextMsg = nextCallback.getNextMessage(); - nextCallback = nextCallback.getNextSendCallback(); - } finally { - msg.getDataBuffer().release(); - } - } - } + @Override + public CompletableFuture getFuture() { + return sendFuture; + } - @Override - public void addCallback(MessageImpl msg, SendCallback scb) { - nextMsg = msg; - nextCallback = scb; + @Override + public SendCallback getNextSendCallback() { + return nextCallback; + } + + @Override + public MessageImpl getNextMessage() { + return nextMsg; + } + + @Override + public void sendComplete(Exception e) { + SendCallback loopingCallback = this; + MessageImpl loopingMsg = currentMsg; + while (loopingCallback != null) { + onSendComplete(e, loopingCallback, loopingMsg); + loopingMsg = loopingCallback.getNextMessage(); + loopingCallback = loopingCallback.getNextSendCallback(); + } + } + + private void onSendComplete(Exception e, SendCallback sendCallback, MessageImpl msg) { + long createdAt = (sendCallback instanceof ProducerImpl.DefaultSendMessageCallback) + ? ((DefaultSendMessageCallback) sendCallback).createdAt : this.createdAt; + long latencyNanos = System.nanoTime() - createdAt; + ByteBuf payload = msg.getDataBuffer(); + if (payload == null) { + log.error("[{}] [{}] Payload is null when calling onSendComplete, which is not expected.", + topic, producerName); + } else { + ReferenceCountUtil.safeRelease(payload); } - }); - return future; + if (e != null) { + stats.incrementSendFailed(); + onSendAcknowledgement(msg, null, e); + sendCallback.getFuture().completeExceptionally(e); + } else { + stats.incrementNumAcksReceived(latencyNanos); + onSendAcknowledgement(msg, msg.getMessageId(), null); + sendCallback.getFuture().complete(msg.getMessageId()); + } + } + + @Override + public void addCallback(MessageImpl msg, SendCallback scb) { + nextMsg = msg; + nextCallback = scb; + } } @Override @@ -454,14 +462,14 @@ public void sendAsync(Message message, SendCallback callback) { // validate msg-size (For batching this will be check at the batch completion size) int compressedSize = compressedPayload.readableBytes(); - if (compressedSize > ClientCnx.getMaxMessageSize() && !this.conf.isChunkingEnabled()) { + if (compressedSize > getMaxMessageSize() && !this.conf.isChunkingEnabled()) { compressedPayload.release(); String compressedStr = conf.getCompressionType() != CompressionType.NONE ? "Compressed" : ""; PulsarClientException.InvalidMessageException invalidMessageException = new PulsarClientException.InvalidMessageException( format("The producer %s of the topic %s sends a %s message with %d bytes that exceeds" + " %d bytes", - producerName, topic, compressedStr, compressedSize, ClientCnx.getMaxMessageSize())); + producerName, topic, compressedStr, compressedSize, getMaxMessageSize())); completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException); return; } @@ -491,19 +499,19 @@ public void sendAsync(Message message, SendCallback callback) { int payloadChunkSize; if (canAddToBatch(msg) || !conf.isChunkingEnabled()) { totalChunks = 1; - payloadChunkSize = ClientCnx.getMaxMessageSize(); + payloadChunkSize = getMaxMessageSize(); } else { // Reserve current metadata size for chunk size to avoid message size overflow. // NOTE: this is not strictly bounded, as metadata will be updated after chunking. // So there is a small chance that the final message size is larger than ClientCnx.getMaxMessageSize(). // But it won't cause produce failure as broker have 10 KB padding space for these cases. - payloadChunkSize = ClientCnx.getMaxMessageSize() - msgMetadata.getSerializedSize(); + payloadChunkSize = getMaxMessageSize() - msgMetadata.getSerializedSize(); if (payloadChunkSize <= 0) { PulsarClientException.InvalidMessageException invalidMessageException = new PulsarClientException.InvalidMessageException( format("The producer %s of the topic %s sends a message with %d bytes metadata that " + "exceeds %d bytes", producerName, topic, - msgMetadata.getSerializedSize(), ClientCnx.getMaxMessageSize())); + msgMetadata.getSerializedSize(), getMaxMessageSize())); completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException); compressedPayload.release(); return; @@ -1659,7 +1667,8 @@ public Iterator iterator() { @Override public CompletableFuture connectionOpened(final ClientCnx cnx) { previousExceptions.clear(); - chunkMaxMessageSize = Math.min(chunkMaxMessageSize, ClientCnx.getMaxMessageSize()); + getConnectionHandler().setMaxMessageSize(cnx.getMaxMessageSize()); + setChunkMaxMessageSize(); final long epoch; synchronized (this) { @@ -2319,11 +2328,11 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from, long e private boolean isMessageSizeExceeded(OpSendMsg op) { if (op.msg != null && !conf.isChunkingEnabled()) { int messageSize = op.getMessageHeaderAndPayloadSize(); - if (messageSize > ClientCnx.getMaxMessageSize()) { + if (messageSize > getMaxMessageSize()) { releaseSemaphoreForSendOp(op); op.sendComplete(new PulsarClientException.InvalidMessageException( format("The producer %s of the topic %s sends a message with %d bytes that exceeds %d bytes", - producerName, topic, messageSize, ClientCnx.getMaxMessageSize()), + producerName, topic, messageSize, getMaxMessageSize()), op.sequenceId)); return true; } @@ -2331,6 +2340,10 @@ private boolean isMessageSizeExceeded(OpSendMsg op) { return false; } + private int getMaxMessageSize() { + return getConnectionHandler().getMaxMessageSize(); + } + public long getDelayInMillis() { OpSendMsg firstMsg = pendingMessages.peek(); if (firstMsg != null) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeSchema.java index 4ae2a21929ad1..62a36fee35147 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeSchema.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.client.impl.schema.generic.MultiVersionGenericProtobufNativeReader.parseProtobufSchema; import com.google.protobuf.Descriptors; +import java.util.Optional; import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.schema.Field; @@ -68,6 +69,11 @@ public Descriptors.Descriptor getProtobufNativeSchema() { return descriptor; } + @Override + public Optional getNativeSchema() { + return Optional.of(descriptor); + } + @Override public boolean supportSchemaVersioning() { return true; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java index 5a223d5da15c0..9995246c175e1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -34,6 +35,7 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Cleanup; @@ -46,6 +48,7 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.conf.TopicConsumerConfigurationData; import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.client.util.ScheduledExecutorProvider; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -263,15 +266,24 @@ public void testTopicPriorityLevel() { assertThat(consumer.getPriorityLevel()).isEqualTo(1); } - @Test(invocationTimeOut = 1000) + @Test public void testSeekAsyncInternal() { // given ClientCnx cnx = mock(ClientCnx.class); CompletableFuture clientReq = new CompletableFuture<>(); when(cnx.sendRequestWithId(any(ByteBuf.class), anyLong())).thenReturn(clientReq); + ScheduledExecutorProvider provider = mock(ScheduledExecutorProvider.class); + ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class); + when(provider.getExecutor()).thenReturn(scheduledExecutorService); + when(consumer.getClient().getScheduledExecutorProvider()).thenReturn(provider); + + CompletableFuture result = consumer.seekAsync(1L); + verify(scheduledExecutorService, atLeast(1)).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + consumer.setClientCnx(cnx); consumer.setState(HandlerState.State.Ready); + consumer.seekStatus.set(ConsumerImpl.SeekStatus.NOT_STARTED); // when CompletableFuture firstResult = consumer.seekAsync(1L); @@ -279,7 +291,6 @@ public void testSeekAsyncInternal() { clientReq.complete(null); - // then assertTrue(firstResult.isDone()); assertTrue(secondResult.isCompletedExceptionally()); verify(cnx, times(1)).sendRequestWithId(any(ByteBuf.class), anyLong()); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdAdvUtilsTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdAdvUtilsTest.java new file mode 100644 index 0000000000000..704dfc9cbd77b --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdAdvUtilsTest.java @@ -0,0 +1,76 @@ +/* + * 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 org.apache.pulsar.client.impl; + +import static org.testng.Assert.assertEquals; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.BitSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Phaser; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import org.apache.pulsar.client.api.MessageIdAdv; +import org.testng.annotations.Test; + +/** + * Unit test for {@link MessageIdAdvUtils}. + */ +public class MessageIdAdvUtilsTest { + + /** + * Call acknowledge concurrently with batch message, and verify that only return true once + * + * @see MessageIdAdvUtils#acknowledge(MessageIdAdv, boolean) + * @see MessageIdAdv#getAckSet() + */ + @Test + public void testAcknowledgeIndividualConcurrently() throws InterruptedException { + ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("pulsar-consumer-%d").build(); + @Cleanup("shutdown") + ExecutorService executorService = Executors.newCachedThreadPool(threadFactory); + for (int i = 0; i < 100; i++) { + int batchSize = 32; + BitSet bitSet = new BitSet(batchSize); + bitSet.set(0, batchSize); + AtomicInteger individualAcked = new AtomicInteger(); + Phaser phaser = new Phaser(1); + CountDownLatch finishLatch = new CountDownLatch(batchSize); + for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) { + phaser.register(); + BatchMessageIdImpl messageId = new BatchMessageIdImpl(1, 0, 0, batchIndex, batchSize, bitSet); + executorService.execute(() -> { + try { + phaser.arriveAndAwaitAdvance(); + if (MessageIdAdvUtils.acknowledge(messageId, true)) { + individualAcked.incrementAndGet(); + } + } finally { + finishLatch.countDown(); + } + }); + } + phaser.arriveAndDeregister(); + finishLatch.await(); + assertEquals(individualAcked.get(), 1); + } + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeReaderTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeReaderTest.java index 4cbb325c82f0c..c358f30ccae73 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeReaderTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericProtobufNativeReaderTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl.schema.generic; +import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.schema.GenericRecord; @@ -29,6 +30,7 @@ import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; @Slf4j public class GenericProtobufNativeReaderTest { @@ -79,6 +81,12 @@ public void testGetNativeRecord() { assertEquals(nativeRecord.getField(nativeRecord.getDescriptorForType().findFieldByName("doubleField")), DOUBLE_FIELD_VLUE); } + @Test + public void testGetNativeSchema() { + assertTrue(genericProtobufNativeSchema.getNativeSchema().isPresent()); + assertTrue(genericProtobufNativeSchema.getNativeSchema().get() instanceof Descriptors.Descriptor); + } + private static final String STRING_FIELD_VLUE = "stringFieldValue"; private static final double DOUBLE_FIELD_VLUE = 0.2D; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java b/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java index 73ecfed0ad059..76d41a7d3d4fc 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java @@ -75,6 +75,8 @@ default int getBatchSize() { * @implNote The message IDs of a batch should share a BitSet. For example, given 3 messages in the same batch whose * size is 3, all message IDs of them should return "111" (i.e. a BitSet whose size is 3 and all bits are 1). If the * 1st message has been acknowledged, the returned BitSet should become "011" (i.e. the 1st bit become 0). + * If the caller performs any read or write operations on the return value of this method, they should do so with + * lock protection. * * @return null if the message is a non-batched message */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceBundleSplitListener.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/LazyLoadableValue.java similarity index 63% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceBundleSplitListener.java rename to pulsar-common/src/main/java/org/apache/pulsar/common/util/LazyLoadableValue.java index a3312f5689e38..063d434a64fc0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceBundleSplitListener.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/LazyLoadableValue.java @@ -16,14 +16,27 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.broker.namespace; +package org.apache.pulsar.common.util; -import java.util.function.Predicate; -import org.apache.pulsar.common.naming.NamespaceBundle; +import java.util.function.Supplier; -/** - * Listener for NamespaceBundle split. +/*** + * Used to lazy load a value, only calculate it when used. Not thread-safety. */ -public interface NamespaceBundleSplitListener extends Predicate { - void onSplit(NamespaceBundle bundle); -} \ No newline at end of file +public class LazyLoadableValue { + + private Supplier loader; + + private T value; + + public LazyLoadableValue(Supplier loader) { + this.loader = loader; + } + + public T getValue() { + if (value == null) { + value = loader.get(); + } + return value; + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index c8c639606aa3e..9bf6302f50f02 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -20,6 +20,7 @@ import io.netty.buffer.ByteBuf; import io.netty.util.CharsetUtil; +import java.nio.CharBuffer; /** * Format strings and numbers into a ByteBuf without any memory allocation. @@ -28,6 +29,7 @@ public class SimpleTextOutputStream { private final ByteBuf buffer; private static final char[] hexChars = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'}; + private final CharBuffer singleCharBuffer = CharBuffer.allocate(1); public SimpleTextOutputStream(ByteBuf buffer) { this.buffer = buffer; @@ -44,11 +46,17 @@ public SimpleTextOutputStream write(byte[] a, int offset, int len) { } public SimpleTextOutputStream write(char c) { - write(String.valueOf(c)); + // In UTF-8, any character from U+0000 to U+007F is encoded in one byte + if (c <= '\u007F') { + buffer.writeByte((byte) c); + return this; + } + singleCharBuffer.put(0, c); + buffer.writeCharSequence(singleCharBuffer, CharsetUtil.UTF_8); return this; } - public SimpleTextOutputStream write(String s) { + public SimpleTextOutputStream write(CharSequence s) { if (s == null) { return this; } @@ -136,4 +144,8 @@ public void write(ByteBuf byteBuf) { public ByteBuf getBuffer() { return buffer; } + + public void writeByte(int b) { + buffer.writeByte(b); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java index 23842fe5b556c..a37628cb300b8 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java @@ -20,12 +20,13 @@ import java.util.BitSet; import java.util.concurrent.locks.StampedLock; -import lombok.EqualsAndHashCode; +import java.util.stream.IntStream; /** - * Safe multithreaded version of {@code BitSet}. + * A {@code BitSet} that is protected by a {@code StampedLock} to provide thread-safe access. + * The {@link #length()} method is not thread safe and is not overridden because StampedLock is not reentrant. + * Use the {@link #safeLength()} method to get the length of the bit set in a thread-safe manner. */ -@EqualsAndHashCode(callSuper = true) public class ConcurrentBitSet extends BitSet { private static final long serialVersionUID = 1L; @@ -39,10 +40,8 @@ public ConcurrentBitSet() { * Creates a bit set whose initial size is large enough to explicitly represent bits with indices in the range * {@code 0} through {@code nbits-1}. All bits are initially {@code false}. * - * @param nbits - * the initial size of the bit set - * @throws NegativeArraySizeException - * if the specified initial size is negative + * @param nbits the initial size of the bit set + * @throws NegativeArraySizeException if the specified initial size is negative */ public ConcurrentBitSet(int nbits) { super(nbits); @@ -65,105 +64,405 @@ public boolean get(int bitIndex) { @Override public void set(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear() { + long stamp = rwLock.writeLock(); + try { + super.clear(); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public int nextSetBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - super.set(bitIndex); + int nextSetBit = super.nextSetBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - super.set(bitIndex); + nextSetBit = super.nextSetBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } + return nextSetBit; } @Override - public void set(int fromIndex, int toIndex) { + public int nextClearBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - super.set(fromIndex, toIndex); + int nextClearBit = super.nextClearBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - super.set(fromIndex, toIndex); + nextClearBit = super.nextClearBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } + return nextClearBit; } @Override - public int nextSetBit(int fromIndex) { + public int previousSetBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - int bit = super.nextSetBit(fromIndex); + int previousSetBit = super.previousSetBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.nextSetBit(fromIndex); + previousSetBit = super.previousSetBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } - return bit; + return previousSetBit; } @Override - public int nextClearBit(int fromIndex) { + public int previousClearBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - int bit = super.nextClearBit(fromIndex); + int previousClearBit = super.previousClearBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.nextClearBit(fromIndex); + previousClearBit = super.previousClearBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } - return bit; + return previousClearBit; } @Override - public int previousSetBit(int fromIndex) { + public boolean isEmpty() { long stamp = rwLock.tryOptimisticRead(); - int bit = super.previousSetBit(fromIndex); + boolean isEmpty = super.isEmpty(); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.previousSetBit(fromIndex); + isEmpty = super.isEmpty(); } finally { rwLock.unlockRead(stamp); } } - return bit; + return isEmpty; } @Override - public int previousClearBit(int fromIndex) { + public int cardinality() { long stamp = rwLock.tryOptimisticRead(); - int bit = super.previousClearBit(fromIndex); + int cardinality = super.cardinality(); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.previousClearBit(fromIndex); + cardinality = super.cardinality(); } finally { rwLock.unlockRead(stamp); } } - return bit; + return cardinality; } @Override - public boolean isEmpty() { + public int size() { long stamp = rwLock.tryOptimisticRead(); - boolean isEmpty = super.isEmpty(); + int size = super.size(); if (!rwLock.validate(stamp)) { // Fallback to read lock stamp = rwLock.readLock(); try { - isEmpty = super.isEmpty(); + size = super.size(); } finally { rwLock.unlockRead(stamp); } } - return isEmpty; + return size; + } + + @Override + public byte[] toByteArray() { + long stamp = rwLock.tryOptimisticRead(); + byte[] byteArray = super.toByteArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + byteArray = super.toByteArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return byteArray; + } + + @Override + public long[] toLongArray() { + long stamp = rwLock.tryOptimisticRead(); + long[] longArray = super.toLongArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + longArray = super.toLongArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return longArray; + } + + @Override + public void flip(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void flip(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int bitIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public BitSet get(int fromIndex, int toIndex) { + long stamp = rwLock.tryOptimisticRead(); + BitSet bitSet = super.get(fromIndex, toIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + bitSet = super.get(fromIndex, toIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return bitSet; + } + + /** + * Thread-safe version of {@code length()}. + * StampedLock is not reentrant and that's why the length() method is not overridden. Overriding length() method + * would require to use a reentrant lock which would be less performant. + * + * @return length of the bit set + */ + public int safeLength() { + long stamp = rwLock.tryOptimisticRead(); + int length = super.length(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + length = super.length(); + } finally { + rwLock.unlockRead(stamp); + } + } + return length; + } + + @Override + public boolean intersects(BitSet set) { + long stamp = rwLock.writeLock(); + try { + return super.intersects(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void and(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.and(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void or(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.or(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void xor(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.xor(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void andNot(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.andNot(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + /** + * Returns the clone of the internal wrapped {@code BitSet}. + * This won't be a clone of the {@code ConcurrentBitSet} object. + * + * @return a clone of the internal wrapped {@code BitSet} + */ + @Override + public Object clone() { + long stamp = rwLock.tryOptimisticRead(); + BitSet clonedBitSet = (BitSet) super.clone(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + clonedBitSet = (BitSet) super.clone(); + } finally { + rwLock.unlockRead(stamp); + } + } + return clonedBitSet; + } + + @Override + public String toString() { + long stamp = rwLock.tryOptimisticRead(); + String str = super.toString(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + str = super.toString(); + } finally { + rwLock.unlockRead(stamp); + } + } + return str; + } + + /** + * This operation is not supported on {@code ConcurrentBitSet}. + */ + @Override + public IntStream stream() { + throw new UnsupportedOperationException("stream is not supported"); + } + + public boolean equals(final Object o) { + if (o == this) { + return true; + } + if (!(o instanceof ConcurrentBitSet)) { + return false; + } + long stamp = rwLock.tryOptimisticRead(); + boolean isEqual = super.equals(o); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + isEqual = super.equals(o); + } finally { + rwLock.unlockRead(stamp); + } + } + return isEqual; + } + + public int hashCode() { + long stamp = rwLock.tryOptimisticRead(); + int hashCode = super.hashCode(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + hashCode = super.hashCode(); + } finally { + rwLock.unlockRead(stamp); + } + } + return hashCode; } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java index f49a6453c72b3..bcff83acd949f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java @@ -19,12 +19,20 @@ package org.apache.pulsar.common.util.netty; import io.netty.resolver.dns.DnsNameResolverBuilder; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; +import java.security.Security; +import java.util.Optional; import lombok.extern.slf4j.Slf4j; @Slf4j public class DnsResolverUtil { + + private static final String CACHE_POLICY_PROP = "networkaddress.cache.ttl"; + private static final String CACHE_POLICY_PROP_FALLBACK = "sun.net.inetaddr.ttl"; + private static final String NEGATIVE_CACHE_POLICY_PROP = "networkaddress.cache.negative.ttl"; + private static final String NEGATIVE_CACHE_POLICY_PROP_FALLBACK = "sun.net.inetaddr.negative.ttl"; + /* default ttl value from sun.net.InetAddressCachePolicy.DEFAULT_POSITIVE, which is used when no security manager + is used */ + private static final int JDK_DEFAULT_TTL = 30; private static final int MIN_TTL = 0; private static final int TTL; private static final int NEGATIVE_TTL; @@ -39,19 +47,35 @@ public class DnsResolverUtil { int ttl = DEFAULT_TTL; int negativeTtl = DEFAULT_NEGATIVE_TTL; try { - // use reflection to call sun.net.InetAddressCachePolicy's get and getNegative methods for getting - // effective JDK settings for DNS caching - Class inetAddressCachePolicyClass = Class.forName("sun.net.InetAddressCachePolicy"); - Method getTTLMethod = inetAddressCachePolicyClass.getMethod("get"); - ttl = (Integer) getTTLMethod.invoke(null); - Method getNegativeTTLMethod = inetAddressCachePolicyClass.getMethod("getNegative"); - negativeTtl = (Integer) getNegativeTTLMethod.invoke(null); - } catch (NoSuchMethodException | ClassNotFoundException | InvocationTargetException - | IllegalAccessException e) { - log.warn("Cannot get DNS TTL settings from sun.net.InetAddressCachePolicy class", e); + String ttlStr = Security.getProperty(CACHE_POLICY_PROP); + if (ttlStr == null) { + // Compatible with sun.net.inetaddr.ttl settings + ttlStr = System.getProperty(CACHE_POLICY_PROP_FALLBACK); + } + String negativeTtlStr = Security.getProperty(NEGATIVE_CACHE_POLICY_PROP); + if (negativeTtlStr == null) { + // Compatible with sun.net.inetaddr.negative.ttl settings + negativeTtlStr = System.getProperty(NEGATIVE_CACHE_POLICY_PROP_FALLBACK); + } + ttl = Optional.ofNullable(ttlStr) + .map(Integer::decode) + .filter(i -> i > 0) + .orElseGet(() -> { + if (System.getSecurityManager() == null) { + return JDK_DEFAULT_TTL; + } + return DEFAULT_TTL; + }); + + negativeTtl = Optional.ofNullable(negativeTtlStr) + .map(Integer::decode) + .filter(i -> i >= 0) + .orElse(DEFAULT_NEGATIVE_TTL); + } catch (NumberFormatException e) { + log.warn("Cannot get DNS TTL settings", e); } - TTL = ttl <= 0 ? DEFAULT_TTL : ttl; - NEGATIVE_TTL = negativeTtl < 0 ? DEFAULT_NEGATIVE_TTL : negativeTtl; + TTL = ttl; + NEGATIVE_TTL = negativeTtl; } private DnsResolverUtil() { diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java index 0ccb960e79887..46599cc45a090 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java @@ -18,13 +18,57 @@ */ package org.apache.pulsar.common.util.netty; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.testng.Assert.assertEquals; import io.netty.channel.EventLoop; import io.netty.resolver.dns.DnsNameResolverBuilder; +import java.security.Security; +import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; public class DnsResolverTest { + private static final int MIN_TTL = 0; + private static final int TTL = 101; + private static final int NEGATIVE_TTL = 121; + private static final String CACHE_POLICY_PROP = "networkaddress.cache.ttl"; + private static final String NEGATIVE_CACHE_POLICY_PROP = "networkaddress.cache.negative.ttl"; + + private String originalCachePolicy; + private String originalNegativeCachePolicy; + + @BeforeClass(alwaysRun = true) + public void beforeClass() { + originalCachePolicy = Security.getProperty(CACHE_POLICY_PROP); + originalNegativeCachePolicy = Security.getProperty(NEGATIVE_CACHE_POLICY_PROP); + Security.setProperty(CACHE_POLICY_PROP, Integer.toString(TTL)); + Security.setProperty(NEGATIVE_CACHE_POLICY_PROP, Integer.toString(NEGATIVE_TTL)); + } + + @AfterClass(alwaysRun = true) + public void afterClass() { + Security.setProperty(CACHE_POLICY_PROP, originalCachePolicy != null ? originalCachePolicy : "-1"); + Security.setProperty(NEGATIVE_CACHE_POLICY_PROP, + originalNegativeCachePolicy != null ? originalNegativeCachePolicy : "0"); + } + + @Test + public void testTTl() { + final DnsNameResolverBuilder builder = mock(DnsNameResolverBuilder.class); + ArgumentCaptor minTtlCaptor = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor maxTtlCaptor = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor negativeTtlCaptor = ArgumentCaptor.forClass(Integer.class); + DnsResolverUtil.applyJdkDnsCacheSettings(builder); + verify(builder).ttl(minTtlCaptor.capture(), maxTtlCaptor.capture()); + verify(builder).negativeTtl(negativeTtlCaptor.capture()); + assertEquals(minTtlCaptor.getValue(), MIN_TTL); + assertEquals(maxTtlCaptor.getValue(), TTL); + assertEquals(negativeTtlCaptor.getValue(), NEGATIVE_TTL); + } @Test public void testMaxTtl() { diff --git a/pulsar-function-go/examples/go.mod b/pulsar-function-go/examples/go.mod index 2b3f1b0c478ce..dfc60e3652276 100644 --- a/pulsar-function-go/examples/go.mod +++ b/pulsar-function-go/examples/go.mod @@ -1,12 +1,61 @@ module github.com/apache/pulsar/pulsar-function-go/examples -go 1.13 +go 1.21 require ( github.com/apache/pulsar-client-go v0.8.1 github.com/apache/pulsar/pulsar-function-go v0.0.0 ) +require ( + github.com/99designs/keyring v1.1.6 // indirect + github.com/AthenZ/athenz v1.10.39 // indirect + github.com/DataDog/zstd v1.5.0 // indirect + github.com/apache/pulsar-client-go/oauth2 v0.0.0-20220120090717-25e59572242e // indirect + github.com/ardielle/ardielle-go v1.5.2 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/danieljoos/wincred v1.0.2 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/dvsekhvalnov/jose2go v1.6.0 // indirect + github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang-jwt/jwt v3.2.2+incompatible // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/golang/snappy v0.0.1 // indirect + github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect + github.com/keybase/go-keychain v0.0.0-20190712205309-48d3d31d256d // indirect + github.com/klauspost/compress v1.10.8 // indirect + github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect + github.com/linkedin/goavro/v2 v2.9.8 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect + github.com/mitchellh/go-homedir v1.1.0 // indirect + github.com/mtibben/percent v0.2.1 // indirect + github.com/pierrec/lz4 v2.0.5+incompatible // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/prometheus/client_golang v1.12.2 // indirect + github.com/prometheus/client_model v0.4.0 // indirect + github.com/prometheus/common v0.32.1 // indirect + github.com/prometheus/procfs v0.7.3 // indirect + github.com/sirupsen/logrus v1.6.0 // indirect + github.com/spaolacci/murmur3 v1.1.0 // indirect + github.com/stretchr/testify v1.8.4 // indirect + go.uber.org/atomic v1.7.0 // indirect + golang.org/x/crypto v0.17.0 // indirect + golang.org/x/net v0.17.0 // indirect + golang.org/x/oauth2 v0.13.0 // indirect + golang.org/x/sys v0.15.0 // indirect + golang.org/x/term v0.15.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/appengine v1.6.8 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect + google.golang.org/grpc v1.60.0 // indirect + google.golang.org/protobuf v1.32.0 // indirect + gopkg.in/yaml.v2 v2.4.0 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect +) + replace github.com/apache/pulsar/pulsar-function-go => ../ replace github.com/apache/pulsar/pulsar-function-go/pf => ../pf diff --git a/pulsar-function-go/examples/go.sum b/pulsar-function-go/examples/go.sum index 0c88121d42346..3fabd79f802db 100644 --- a/pulsar-function-go/examples/go.sum +++ b/pulsar-function-go/examples/go.sum @@ -71,8 +71,10 @@ github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kB github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -90,8 +92,9 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dimfeld/httptreemux v5.0.1+incompatible h1:Qj3gVcDNoOthBAqftuD596rm4wg/adLLz5xh5CmpiCA= github.com/dimfeld/httptreemux v5.0.1+incompatible/go.mod h1:rbUlSV+CCpv/SuqUTP/8Bk2O3LyUV436/yaRGkhP6Z0= -github.com/dvsekhvalnov/jose2go v0.0.0-20200901110807-248326c1351b h1:HBah4D48ypg3J7Np4N+HY/ZR76fx3HEUGxDU6Uk39oQ= github.com/dvsekhvalnov/jose2go v0.0.0-20200901110807-248326c1351b/go.mod h1:7BvyPhdbLxMXIYTFPLsyJRFMsKmOZnQmzh6Gb+uquuM= +github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA2EjTY= +github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -152,8 +155,9 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -168,8 +172,9 @@ github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= +github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= @@ -186,8 +191,9 @@ github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= @@ -226,6 +232,7 @@ github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= @@ -243,7 +250,6 @@ github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxv github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= @@ -270,10 +276,13 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs= github.com/mtibben/percent v0.2.1/go.mod h1:KG9uO+SZkUp+VkRHsCdYQV3XSZrrSpR3O9ibNBTZrns= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -299,22 +308,27 @@ github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndr github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.1 h1:+4eQaD7vAZ6DsfsxB15hbE0odUjGI5ARs9yskGu1v4s= +github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_golang v1.11.1/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= +github.com/prometheus/client_golang v1.12.2 h1:51L9cDoUHVrXx4zWYlcLQIZ+d+VXHgqnYKkIuq4g/34= +github.com/prometheus/client_golang v1.12.2/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= +github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= +github.com/prometheus/common v0.32.1 h1:hWIdL3N2HoUx3B8j3YN9mWor0qhY/NlEKZEaXxuIRh4= +github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0VU= +github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= @@ -337,21 +351,24 @@ github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.2.0 h1:Hbg2NidpLE8veEBkEZTL3CvlkUIVzuU9jDplZO54c48= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= +github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= go.etcd.io/etcd/client/v2 v2.305.0/go.mod h1:h9puh54ZTgAKtEbut2oe9P4L/oqKCVB6xsXlzd7alYQ= @@ -374,8 +391,10 @@ golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -411,6 +430,7 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -451,8 +471,11 @@ golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985 h1:4CSI6oo7cOjJKajidEljs9h+uP0rRZBPPPhcCbj5mw8= +golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -464,8 +487,10 @@ golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602 h1:0Ja1LBD+yisY6RWM/BH7TJVXWsSjs2VwBSmvSX4HdBc= golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.13.0 h1:jDDenyj+WgFtmV3zYVoi8aE2BwtXFLWOA67ZfNWftiY= +golang.org/x/oauth2 v0.13.0/go.mod h1:/JMhi4ZRXAf4HG9LiNmxvk+45+96RUlVThiH8FzNBn0= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -477,6 +502,7 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -531,9 +557,17 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40 h1:JWgyZ1qgdTaF3N3oxC+MdTV7qvEEgHo3otj+HB5CM7Q= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= +golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -541,8 +575,11 @@ golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -597,6 +634,7 @@ golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -630,8 +668,9 @@ google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= +google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -672,8 +711,9 @@ google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6D google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= -google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c h1:wtujag7C+4D6KMoulW9YauvK2lgdvCMS260jsqqBXr0= google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 h1:6GQBEOdGkX6MMTLT9V+TjtIRZCw9VPD5Z+yHY9wMgS0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97/go.mod h1:v7nGkzlmW8P3n/bKmWBn2WpBjpOEx8Q6gMueudAmKfY= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -693,8 +733,9 @@ google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA5 google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.38.0 h1:/9BgsAsa5nWe26HqOlvlgJnqBuktYOLCgjCPqsa56W0= google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.60.0 h1:6FQAR0kM31P6MRdeluor2w2gPaS4SVNrD/DNTxrQ15k= +google.golang.org/grpc v1.60.0/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -706,13 +747,15 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= +google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= +gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= @@ -730,8 +773,9 @@ gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/pulsar-function-go/go.mod b/pulsar-function-go/go.mod index 9d6031a93a0ff..aa1a081842508 100644 --- a/pulsar-function-go/go.mod +++ b/pulsar-function-go/go.mod @@ -1,19 +1,60 @@ module github.com/apache/pulsar/pulsar-function-go -go 1.13 +go 1.21 require ( github.com/apache/pulsar-client-go v0.8.1 - github.com/golang/protobuf v1.5.2 + github.com/golang/protobuf v1.5.3 github.com/prometheus/client_golang v1.12.2 - github.com/prometheus/client_model v0.2.0 + github.com/prometheus/client_model v0.4.0 github.com/sirupsen/logrus v1.6.0 - github.com/stretchr/testify v1.7.0 - google.golang.org/grpc v1.38.0 - google.golang.org/protobuf v1.26.0 + github.com/stretchr/testify v1.8.4 + google.golang.org/grpc v1.60.0 + google.golang.org/protobuf v1.32.0 gopkg.in/yaml.v2 v2.4.0 ) +require ( + github.com/99designs/keyring v1.1.6 // indirect + github.com/AthenZ/athenz v1.10.39 // indirect + github.com/DataDog/zstd v1.5.0 // indirect + github.com/apache/pulsar-client-go/oauth2 v0.0.0-20220120090717-25e59572242e // indirect + github.com/ardielle/ardielle-go v1.5.2 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/danieljoos/wincred v1.0.2 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/dvsekhvalnov/jose2go v1.6.0 // indirect + github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang-jwt/jwt v3.2.2+incompatible // indirect + github.com/golang/snappy v0.0.1 // indirect + github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect + github.com/keybase/go-keychain v0.0.0-20190712205309-48d3d31d256d // indirect + github.com/klauspost/compress v1.10.8 // indirect + github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect + github.com/linkedin/goavro/v2 v2.9.8 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect + github.com/mitchellh/go-homedir v1.1.0 // indirect + github.com/mtibben/percent v0.2.1 // indirect + github.com/pierrec/lz4 v2.0.5+incompatible // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/prometheus/common v0.32.1 // indirect + github.com/prometheus/procfs v0.7.3 // indirect + github.com/spaolacci/murmur3 v1.1.0 // indirect + go.uber.org/atomic v1.7.0 // indirect + golang.org/x/crypto v0.17.0 // indirect + golang.org/x/net v0.17.0 // indirect + golang.org/x/oauth2 v0.13.0 // indirect + golang.org/x/sys v0.15.0 // indirect + golang.org/x/term v0.15.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/appengine v1.6.8 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect +) + replace github.com/apache/pulsar/pulsar-function-go/pf => ./pf replace github.com/apache/pulsar/pulsar-function-go/logutil => ./logutil diff --git a/pulsar-function-go/go.sum b/pulsar-function-go/go.sum index d5fddca34e5c2..3fabd79f802db 100644 --- a/pulsar-function-go/go.sum +++ b/pulsar-function-go/go.sum @@ -72,8 +72,9 @@ github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqO github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -91,8 +92,9 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dimfeld/httptreemux v5.0.1+incompatible h1:Qj3gVcDNoOthBAqftuD596rm4wg/adLLz5xh5CmpiCA= github.com/dimfeld/httptreemux v5.0.1+incompatible/go.mod h1:rbUlSV+CCpv/SuqUTP/8Bk2O3LyUV436/yaRGkhP6Z0= -github.com/dvsekhvalnov/jose2go v0.0.0-20200901110807-248326c1351b h1:HBah4D48ypg3J7Np4N+HY/ZR76fx3HEUGxDU6Uk39oQ= github.com/dvsekhvalnov/jose2go v0.0.0-20200901110807-248326c1351b/go.mod h1:7BvyPhdbLxMXIYTFPLsyJRFMsKmOZnQmzh6Gb+uquuM= +github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA2EjTY= +github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -153,8 +155,9 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -169,8 +172,9 @@ github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= +github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= @@ -187,8 +191,9 @@ github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= @@ -245,7 +250,6 @@ github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxv github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= @@ -277,6 +281,8 @@ github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs github.com/mtibben/percent v0.2.1/go.mod h1:KG9uO+SZkUp+VkRHsCdYQV3XSZrrSpR3O9ibNBTZrns= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -309,8 +315,9 @@ github.com/prometheus/client_golang v1.12.2/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrb github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= +github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= @@ -344,21 +351,24 @@ github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.2.0 h1:Hbg2NidpLE8veEBkEZTL3CvlkUIVzuU9jDplZO54c48= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= +github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= go.etcd.io/etcd/client/v2 v2.305.0/go.mod h1:h9puh54ZTgAKtEbut2oe9P4L/oqKCVB6xsXlzd7alYQ= @@ -381,8 +391,10 @@ golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -418,6 +430,7 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -459,8 +472,10 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985 h1:4CSI6oo7cOjJKajidEljs9h+uP0rRZBPPPhcCbj5mw8= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -473,8 +488,9 @@ golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c h1:pkQiBZBvdos9qq4wBAHqlzuZHEXo07pqV06ef90u1WI= golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.13.0 h1:jDDenyj+WgFtmV3zYVoi8aE2BwtXFLWOA67ZfNWftiY= +golang.org/x/oauth2 v0.13.0/go.mod h1:/JMhi4ZRXAf4HG9LiNmxvk+45+96RUlVThiH8FzNBn0= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -486,6 +502,7 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -541,9 +558,16 @@ golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220114195835-da31bd327af9 h1:XfKQ4OlFl8okEOr5UvAqFRVj8pY/4yfcXrddB8qAbU0= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= +golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -551,8 +575,11 @@ golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -607,6 +634,7 @@ golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -640,8 +668,9 @@ google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= +google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -682,8 +711,9 @@ google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6D google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= -google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c h1:wtujag7C+4D6KMoulW9YauvK2lgdvCMS260jsqqBXr0= google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 h1:6GQBEOdGkX6MMTLT9V+TjtIRZCw9VPD5Z+yHY9wMgS0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97/go.mod h1:v7nGkzlmW8P3n/bKmWBn2WpBjpOEx8Q6gMueudAmKfY= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -703,8 +733,9 @@ google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA5 google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.38.0 h1:/9BgsAsa5nWe26HqOlvlgJnqBuktYOLCgjCPqsa56W0= google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.60.0 h1:6FQAR0kM31P6MRdeluor2w2gPaS4SVNrD/DNTxrQ15k= +google.golang.org/grpc v1.60.0/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -716,13 +747,15 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= +google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= +gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= @@ -740,8 +773,9 @@ gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/pulsar-function-go/pf/context.go b/pulsar-function-go/pf/context.go index fd3da9b59b656..0b269a568ffec 100644 --- a/pulsar-function-go/pf/context.go +++ b/pulsar-function-go/pf/context.go @@ -54,14 +54,14 @@ func NewFuncContext() *FunctionContext { return fc } -//GetInstanceID returns the id of the instance that invokes the running pulsar -//function. +// GetInstanceID returns the id of the instance that invokes the running pulsar +// function. func (c *FunctionContext) GetInstanceID() int { return c.instanceConf.instanceID } -//GetInputTopics returns a list of all input topics the pulsar function has been -//invoked on +// GetInputTopics returns a list of all input topics the pulsar function has been +// invoked on func (c *FunctionContext) GetInputTopics() []string { inputMap := c.instanceConf.funcDetails.GetSource().InputSpecs inputTopics := make([]string, len(inputMap)) @@ -73,84 +73,84 @@ func (c *FunctionContext) GetInputTopics() []string { return inputTopics } -//GetOutputTopic returns the output topic the pulsar function was invoked on +// GetOutputTopic returns the output topic the pulsar function was invoked on func (c *FunctionContext) GetOutputTopic() string { return c.instanceConf.funcDetails.GetSink().Topic } -//GetTenantAndNamespace returns the tenant and namespace the pulsar function -//belongs to in the format of `/` +// GetTenantAndNamespace returns the tenant and namespace the pulsar function +// belongs to in the format of `/` func (c *FunctionContext) GetTenantAndNamespace() string { return c.GetFuncTenant() + "/" + c.GetFuncNamespace() } -//GetTenantAndNamespaceAndName returns the full name of the pulsar function in -//the format of `//` +// GetTenantAndNamespaceAndName returns the full name of the pulsar function in +// the format of `//` func (c *FunctionContext) GetTenantAndNamespaceAndName() string { return c.GetFuncTenant() + "/" + c.GetFuncNamespace() + "/" + c.GetFuncName() } -//GetFuncTenant returns the tenant the pulsar function belongs to +// GetFuncTenant returns the tenant the pulsar function belongs to func (c *FunctionContext) GetFuncTenant() string { return c.instanceConf.funcDetails.Tenant } -//GetFuncName returns the name given to the pulsar function +// GetFuncName returns the name given to the pulsar function func (c *FunctionContext) GetFuncName() string { return c.instanceConf.funcDetails.Name } -//GetFuncNamespace returns the namespace the pulsar function belongs to +// GetFuncNamespace returns the namespace the pulsar function belongs to func (c *FunctionContext) GetFuncNamespace() string { return c.instanceConf.funcDetails.Namespace } -//GetFuncID returns the id of the pulsar function +// GetFuncID returns the id of the pulsar function func (c *FunctionContext) GetFuncID() string { return c.instanceConf.funcID } -//GetPort returns the port the pulsar function communicates on +// GetPort returns the port the pulsar function communicates on func (c *FunctionContext) GetPort() int { return c.instanceConf.port } -//GetClusterName returns the name of the cluster the pulsar function is running -//in +// GetClusterName returns the name of the cluster the pulsar function is running +// in func (c *FunctionContext) GetClusterName() string { return c.instanceConf.clusterName } -//GetExpectedHealthCheckInterval returns the expected time between health checks -//in seconds +// GetExpectedHealthCheckInterval returns the expected time between health checks +// in seconds func (c *FunctionContext) GetExpectedHealthCheckInterval() int32 { return c.instanceConf.expectedHealthCheckInterval } -//GetExpectedHealthCheckIntervalAsDuration returns the expected time between -//health checks in seconds as a time.Duration +// GetExpectedHealthCheckIntervalAsDuration returns the expected time between +// health checks in seconds as a time.Duration func (c *FunctionContext) GetExpectedHealthCheckIntervalAsDuration() time.Duration { return time.Duration(c.instanceConf.expectedHealthCheckInterval) } -//GetMaxIdleTime returns the amount of time the pulsar function has to respond -//to the most recent health check before it is considered to be failing. +// GetMaxIdleTime returns the amount of time the pulsar function has to respond +// to the most recent health check before it is considered to be failing. func (c *FunctionContext) GetMaxIdleTime() int64 { return int64(c.GetExpectedHealthCheckIntervalAsDuration() * 3 * time.Second) } -//GetFuncVersion returns the version of the pulsar function +// GetFuncVersion returns the version of the pulsar function func (c *FunctionContext) GetFuncVersion() string { return c.instanceConf.funcVersion } -//GetUserConfValue returns the value of a key from the pulsar function's user -//configuration map +// GetUserConfValue returns the value of a key from the pulsar function's user +// configuration map func (c *FunctionContext) GetUserConfValue(key string) interface{} { return c.userConfigs[key] } -//GetUserConfMap returns the pulsar function's user configuration map +// GetUserConfMap returns the pulsar function's user configuration map func (c *FunctionContext) GetUserConfMap() map[string]interface{} { return c.userConfigs } @@ -172,12 +172,12 @@ func (c *FunctionContext) GetCurrentRecord() pulsar.Message { return c.record } -//GetMetricsPort returns the port the pulsar function metrics listen on +// GetMetricsPort returns the port the pulsar function metrics listen on func (c *FunctionContext) GetMetricsPort() int { return c.instanceConf.metricsPort } -//RecordMetric records an observation to the user_metric summary with the provided value +// RecordMetric records an observation to the user_metric summary with the provided value func (c *FunctionContext) RecordMetric(metricName string, metricValue float64) { v, ok := c.userMetrics.Load(metricName) if !ok { diff --git a/pulsar-function-go/pf/function.go b/pulsar-function-go/pf/function.go index 4efbb2b4cec1c..f6e46ff7ac83b 100644 --- a/pulsar-function-go/pf/function.go +++ b/pulsar-function-go/pf/function.go @@ -143,25 +143,25 @@ func newFunction(inputFunc interface{}) function { } // Rules: -// -// * handler must be a function -// * handler may take between 0 and two arguments. -// * if there are two arguments, the first argument must satisfy the "context.Context" interface. -// * handler may return between 0 and two arguments. -// * if there are two return values, the second argument must be an error. -// * if there is one return value it must be an error. + +// - handler must be a function +// - handler may take between 0 and two arguments. +// - if there are two arguments, the first argument must satisfy the "context.Context" interface. +// - handler may return between 0 and two arguments. +// - if there are two return values, the second argument must be an error. +// - if there is one return value it must be an error. // // Valid function signatures: // -// func () -// func () error -// func (input) error -// func () (output, error) -// func (input) (output, error) -// func (context.Context) error -// func (context.Context, input) error -// func (context.Context) (output, error) -// func (context.Context, input) (output, error) +// func () +// func () error +// func (input) error +// func () (output, error) +// func (input) (output, error) +// func (context.Context) error +// func (context.Context, input) error +// func (context.Context) (output, error) +// func (context.Context, input) (output, error) // // Where "input" and "output" are types compatible with the "encoding/json" standard library. // See https://golang.org/pkg/encoding/json/#Unmarshal for how deserialization behaves diff --git a/pulsar-function-go/pf/instanceControlServicer_test.go b/pulsar-function-go/pf/instanceControlServicer_test.go index 9344d0a591547..30021adfb926c 100644 --- a/pulsar-function-go/pf/instanceControlServicer_test.go +++ b/pulsar-function-go/pf/instanceControlServicer_test.go @@ -30,6 +30,7 @@ import ( "github.com/golang/protobuf/ptypes/empty" "github.com/stretchr/testify/assert" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/test/bufconn" ) @@ -62,7 +63,8 @@ func TestInstanceControlServicer_serve_creates_valid_instance(t *testing.T) { // Now we can setup the client: ctx := context.Background() - conn, err := grpc.DialContext(ctx, "bufnet", grpc.WithContextDialer(getBufDialer(lis)), grpc.WithInsecure()) + conn, err := grpc.DialContext(ctx, "bufnet", grpc.WithContextDialer(getBufDialer(lis)), + grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { t.Fatalf("Failed to dial bufnet: %v", err) } @@ -86,7 +88,7 @@ func instanceCommunicationClient(t *testing.T, instance *goInstance) pb.Instance } var ( - ctx context.Context = context.Background() + ctx = context.Background() cf context.CancelFunc ) @@ -119,7 +121,8 @@ func instanceCommunicationClient(t *testing.T, instance *goInstance) pb.Instance }() // Now we can setup the client: - conn, err := grpc.DialContext(ctx, "bufnet", grpc.WithContextDialer(getBufDialer(lis)), grpc.WithInsecure()) + conn, err := grpc.DialContext(ctx, "bufnet", grpc.WithContextDialer(getBufDialer(lis)), + grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { t.Fatalf("Failed to dial bufnet: %v", err) } diff --git a/pulsar-function-go/pf/stats.go b/pulsar-function-go/pf/stats.go index 85a7fff8a38bb..d25424b37047a 100644 --- a/pulsar-function-go/pf/stats.go +++ b/pulsar-function-go/pf/stats.go @@ -282,7 +282,7 @@ func (stat *StatWithLabelValues) addUserException(err error) { stat.reportUserExceptionPrometheus(err) } -//@limits(calls=5, period=60) +// @limits(calls=5, period=60) func (stat *StatWithLabelValues) reportUserExceptionPrometheus(exception error) { errorTS := []string{exception.Error()} exceptionMetricLabels := append(stat.metricsLabels, errorTS...) @@ -312,7 +312,7 @@ func (stat *StatWithLabelValues) addSysException(exception error) { stat.reportSystemExceptionPrometheus(exception) } -//@limits(calls=5, period=60) +// @limits(calls=5, period=60) func (stat *StatWithLabelValues) reportSystemExceptionPrometheus(exception error) { errorTS := []string{exception.Error()} exceptionMetricLabels := append(stat.metricsLabels, errorTS...) diff --git a/pulsar-function-go/pf/stats_test.go b/pulsar-function-go/pf/stats_test.go index 7b415ef5eff0b..0921038bba835 100644 --- a/pulsar-function-go/pf/stats_test.go +++ b/pulsar-function-go/pf/stats_test.go @@ -22,16 +22,16 @@ package pf import ( "context" "fmt" - "io/ioutil" + "io" "math" "net/http" "testing" "time" - "github.com/golang/protobuf/proto" "github.com/golang/protobuf/ptypes/empty" "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/assert" + "google.golang.org/protobuf/encoding/prototext" prometheus_client "github.com/prometheus/client_model/go" ) @@ -77,73 +77,76 @@ func TestExampleSummaryVec(t *testing.T) { expectedValue := "name: \"pond_temperature_celsius\"\n" + "help: \"The temperature of the frog pond.\"\n" + "type: SUMMARY\n" + - "metric: <\n" + - " label: <\n" + + "metric: {\n" + + " label: {\n" + " name: \"species\"\n" + " value: \"leiopelma-hochstetteri\"\n" + - " >\n" + - " summary: <\n" + + " }\n" + + " summary: {\n" + " sample_count: 0\n" + " sample_sum: 0\n" + - " quantile: <\n" + + " quantile: {\n" + " quantile: 0.5\n" + " value: nan\n" + - " >\n" + - " quantile: <\n" + + " }\n" + + " quantile: {\n" + " quantile: 0.9\n" + " value: nan\n" + - " >\n" + - " quantile: <\n" + + " }\n" + + " quantile: {\n" + " quantile: 0.99\n" + " value: nan\n" + - " >\n" + - " >\n" + - ">\n" + - "metric: <\n" + - " label: <\n" + + " }\n" + + " }\n" + + "}\n" + + "metric: {\n" + + " label: {\n" + " name: \"species\"\n" + " value: \"lithobates-catesbeianus\"\n" + - " >\n" + - " summary: <\n" + + " }\n" + + " summary: {\n" + " sample_count: 1000\n" + " sample_sum: 31956.100000000017\n" + - " quantile: <\n" + + " quantile: {\n" + " quantile: 0.5\n" + " value: 32.4\n" + - " >\n" + - " quantile: <\n" + + " }\n" + + " quantile: {\n" + " quantile: 0.9\n" + " value: 41.4\n" + - " >\n" + - " quantile: <\n" + + " }\n" + + " quantile: {\n" + " quantile: 0.99\n" + " value: 41.9\n" + - " >\n" + - " >\n" + - ">\n" + - "metric: <\n" + - " label: <\n" + + " }\n" + + " }\n" + + "}\n" + + "metric: {\n" + + " label: {\n" + " name: \"species\"\n" + " value: \"litoria-caerulea\"\n" + - " >\n" + - " summary: <\n" + + " }\n" + + " summary: {\n" + " sample_count: 1000\n" + " sample_sum: 29969.50000000001\n" + - " quantile: <\n" + + " quantile: {\n" + " quantile: 0.5\n" + " value: 31.1\n" + - " >\n" + - " quantile: <\n" + + " }\n" + + " quantile: {\n" + " quantile: 0.9\n" + " value: 41.3\n" + - " >\n" + - " quantile: <\n" + + " }\n" + + " quantile: {\n" + " quantile: 0.99\n" + " value: 41.9\n" + - " >\n" + - " >\n" + - ">\n" - assert.Equal(t, expectedValue, proto.MarshalTextString(metricFamilies[0])) + " }\n" + + " }\n" + + "}\n" + + r, err := prototext.MarshalOptions{Indent: " "}.Marshal(metricFamilies[0]) + assert.NoError(t, err) + assert.Equal(t, expectedValue, string(r)) } func TestExampleSummaryVec_Pulsar(t *testing.T) { _statProcessLatencyMs1 := prometheus.NewSummaryVec( @@ -202,7 +205,7 @@ func TestMetricsServer(t *testing.T) { assert.Equal(t, nil, err) assert.NotEqual(t, nil, resp) assert.Equal(t, 200, resp.StatusCode) - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) assert.Equal(t, nil, err) assert.NotEmpty(t, body) resp.Body.Close() @@ -211,7 +214,7 @@ func TestMetricsServer(t *testing.T) { assert.Equal(t, nil, err) assert.NotEqual(t, nil, resp) assert.Equal(t, 200, resp.StatusCode) - body, err = ioutil.ReadAll(resp.Body) + body, err = io.ReadAll(resp.Body) assert.Equal(t, nil, err) assert.NotEmpty(t, body) resp.Body.Close() @@ -229,7 +232,7 @@ func TestUserMetrics(t *testing.T) { assert.Equal(t, nil, err) assert.NotEqual(t, nil, resp) assert.Equal(t, 200, resp.StatusCode) - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) assert.Equal(t, nil, err) assert.NotEmpty(t, body) assert.NotContainsf(t, string(body), "pulsar_function_user_metric", "user metric should not appear yet") @@ -245,7 +248,7 @@ func TestUserMetrics(t *testing.T) { assert.Equal(t, nil, err) assert.NotEqual(t, nil, resp) assert.Equal(t, 200, resp.StatusCode) - body, err = ioutil.ReadAll(resp.Body) + body, err = io.ReadAll(resp.Body) assert.Equal(t, nil, err) assert.NotEmpty(t, body) diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index 84b943e5671ac..255ed5f3218a0 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -224,7 +224,7 @@ private static URI initializeStandaloneWorkerService(PulsarClientCreator clientC log.warn("Retry to connect to Pulsar service at {}", workerConfig.getPulsarWebServiceUrl()); if (retries >= maxRetries) { log.error("Failed to connect to Pulsar service at {} after {} attempts", - workerConfig.getPulsarFunctionsNamespace(), maxRetries); + workerConfig.getPulsarFunctionsNamespace(), maxRetries, e); throw e; } retries++; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index 6d07e5870917a..ea5517e0fd4eb 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -1080,7 +1080,7 @@ public String triggerFunction(final String tenant, try { worker().getBrokerAdmin().topics().getSubscriptions(inputTopicToWrite); } catch (PulsarAdminException e) { - log.error("Function in trigger function is not ready @ /{}/{}/{}", tenant, namespace, functionName); + log.error("Function in trigger function is not ready @ /{}/{}/{}", tenant, namespace, functionName, e); throw new RestException(Status.BAD_REQUEST, "Function in trigger function is not ready"); } String outputTopic = functionMetaData.getFunctionDetails().getSink().getTopic(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java index a075d3e18a0b3..4cbd7c8cbcb12 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java @@ -169,7 +169,7 @@ public void registerFunction(final String tenant, worker().getFunctionRuntimeManager().getRuntimeFactory().doAdmissionChecks(functionDetails); } catch (Exception e) { log.error("{} {}/{}/{} cannot be admitted by the runtime factory", - ComponentTypeUtils.toString(componentType), tenant, namespace, functionName); + ComponentTypeUtils.toString(componentType), tenant, namespace, functionName, e); throw new RestException(Response.Status.BAD_REQUEST, String.format("%s %s cannot be admitted:- %s", ComponentTypeUtils.toString(componentType), functionName, e.getMessage())); } @@ -327,7 +327,7 @@ public void updateFunction(final String tenant, worker().getFunctionRuntimeManager().getRuntimeFactory().doAdmissionChecks(functionDetails); } catch (Exception e) { log.error("Updated {} {}/{}/{} cannot be submitted to runtime factory", - ComponentTypeUtils.toString(componentType), tenant, namespace, functionName); + ComponentTypeUtils.toString(componentType), tenant, namespace, functionName, e); throw new RestException(Response.Status.BAD_REQUEST, String.format("%s %s cannot be admitted:- %s", ComponentTypeUtils.toString(componentType), functionName, e.getMessage())); } diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 1ca3eb4f7138d..ca72d24f8d638 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -29,9 +29,10 @@ - 2.7.3 + 2.9.3 4.1.11 1.37.0 + 4.1.100.Final pulsar-io-alluxio @@ -56,12 +57,6 @@ org.alluxio alluxio-core-client-fs ${alluxio.version} - - - grpc-netty - io.grpc - - @@ -74,10 +69,6 @@ org.glassfish javax.el - - grpc-netty - io.grpc - @@ -90,22 +81,32 @@ com.google.guava guava - - - - io.grpc - grpc-netty - ${grpc.version} - - - - io.dropwizard.metrics - metrics-jvm - ${metrics.version} - - + + + + io.netty + netty-bom + ${netty.version} + pom + import + + + io.grpc + grpc-bom + ${grpc.version} + pom + import + + + io.dropwizard.metrics + metrics-jvm + ${metrics.version} + + + + diff --git a/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java b/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java index 413f05e0e17c5..3b72dc9666b78 100644 --- a/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java +++ b/pulsar-io/alluxio/src/main/java/org/apache/pulsar/io/alluxio/sink/AlluxioSink.java @@ -22,12 +22,13 @@ import alluxio.client.WriteType; import alluxio.client.file.FileOutStream; import alluxio.client.file.FileSystem; +import alluxio.conf.Configuration; import alluxio.conf.InstancedConfiguration; import alluxio.conf.PropertyKey; import alluxio.exception.AlluxioException; import alluxio.grpc.CreateFilePOptions; import alluxio.grpc.WritePType; -import alluxio.util.FileSystemOptions; +import alluxio.util.FileSystemOptionsUtils; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -78,7 +79,7 @@ public class AlluxioSink implements Sink { private AlluxioSinkConfig alluxioSinkConfig; private AlluxioState alluxioState; - private InstancedConfiguration configuration = InstancedConfiguration.defaults(); + private InstancedConfiguration configuration = Configuration.modifiableGlobal(); private ObjectMapper objectMapper = new ObjectMapper(); @@ -205,7 +206,7 @@ private void writeToAlluxio(Record record) throws AlluxioExceptio private void createTmpFile() throws AlluxioException, IOException { CreateFilePOptions.Builder optionsBuilder = - FileSystemOptions.createFileDefaults(configuration).toBuilder(); + FileSystemOptionsUtils.createFileDefaults(configuration).toBuilder(); UUID id = UUID.randomUUID(); String fileExtension = alluxioSinkConfig.getFileExtension(); tmpFilePath = tmpFileDirPath + "/" + id.toString() + "_tmp" + fileExtension; diff --git a/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java b/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java index 9325a2255ab0a..bf40581aae155 100644 --- a/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java +++ b/pulsar-io/alluxio/src/test/java/org/apache/pulsar/io/alluxio/sink/AlluxioSinkTest.java @@ -22,8 +22,8 @@ import alluxio.client.WriteType; import alluxio.client.file.FileSystem; import alluxio.client.file.URIStatus; +import alluxio.conf.Configuration; import alluxio.conf.PropertyKey; -import alluxio.conf.ServerConfiguration; import alluxio.master.LocalAlluxioCluster; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.FilenameUtils; @@ -237,8 +237,8 @@ public Object getNativeObject() { private LocalAlluxioCluster setupSingleMasterCluster() throws Exception { // Setup and start the local alluxio cluster LocalAlluxioCluster cluster = new LocalAlluxioCluster(); - cluster.initConfiguration(getTestName(getClass().getSimpleName(), LocalAlluxioCluster.DEFAULT_TEST_NAME)); - ServerConfiguration.set(PropertyKey.USER_FILE_WRITE_TYPE_DEFAULT, WriteType.MUST_CACHE); + cluster.initConfiguration(getTestName(getClass().getSimpleName(), "test")); + Configuration.set(PropertyKey.USER_FILE_WRITE_TYPE_DEFAULT, WriteType.MUST_CACHE); cluster.start(); return cluster; } diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index 86735e9d4f196..8674c931ee270 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -45,6 +45,10 @@ com.fasterxml.jackson.core jackson-databind + + org.apache.commons + commons-collections4 + com.fasterxml.jackson.dataformat @@ -65,6 +69,10 @@ avro org.apache.avro + + commons-collections + commons-collections + @@ -90,6 +98,10 @@ io.netty netty + + commons-collections + commons-collections + diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 5c0adf6be27de..b0e5a3d0b3e8d 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -71,6 +71,13 @@ provided + + ${project.groupId} + pulsar-client-original + ${project.version} + test + + \ No newline at end of file diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java index 36c3674091932..3655688c0f3ad 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.api.schema.KeyValueSchema; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.jdbc.JdbcUtils.ColumnId; @@ -137,6 +138,10 @@ public Mutation createMutation(Record message) { } recordValueGetter = (k) -> data.get(k); } else { + SchemaType schemaType = message.getSchema().getSchemaInfo().getType(); + if (schemaType.isPrimitive()) { + throw new UnsupportedOperationException("Primitive schema is not supported: " + schemaType); + } recordValueGetter = (key) -> ((GenericRecord) record).getField(key); } String action = message.getProperties().get(ACTION_PROPERTY); diff --git a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java index b15eb832242c7..c088dd3c42c32 100644 --- a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java +++ b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java @@ -22,6 +22,10 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.util.Utf8; +import org.apache.pulsar.client.api.schema.GenericObject; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; +import org.apache.pulsar.functions.api.Record; import org.testng.Assert; import org.testng.annotations.Test; @@ -143,5 +147,26 @@ private Schema createFieldAndGetSchema(Function record = new Record() { + @Override + public org.apache.pulsar.client.api.Schema getSchema() { + return autoConsumeSchema; + } + + @Override + public GenericRecord getValue() { + return null; + } + }; + baseJdbcAutoSchemaSink.createMutation((Record) record); + } + } \ No newline at end of file diff --git a/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java b/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java index d9ed4cbd442bf..ca01615bef193 100644 --- a/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java +++ b/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java @@ -48,6 +48,7 @@ import org.apache.pulsar.client.api.schema.RecordSchemaBuilder; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.schema.AvroSchema; import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; import org.apache.pulsar.common.schema.KeyValue; @@ -282,9 +283,12 @@ public void TestUnknownAction() throws Exception { } @Test + @SuppressWarnings("unchecked") public void TestUpdateAction() throws Exception { AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); Foo updateObj = new Foo(); updateObj.setField1("ValueOfField3"); @@ -292,10 +296,11 @@ public void TestUpdateAction() throws Exception { updateObj.setField3(4); byte[] updateBytes = schema.encode(updateObj); - Message updateMessage = mock(MessageImpl.class); + Message updateMessage = mock(MessageImpl.class); CompletableFuture future = new CompletableFuture<>(); - Record updateRecord = PulsarRecord.builder() + Record updateRecord = PulsarRecord.builder() .message(updateMessage) + .schema(autoConsumeSchema) .topicName("fake_topic_name") .ackFunction(() -> future.complete(null)) .build(); @@ -312,7 +317,7 @@ public void TestUpdateAction() throws Exception { updateMessage.getValue().toString(), updateRecord.getValue().toString()); - jdbcSink.write(updateRecord); + jdbcSink.write((Record) updateRecord); future.get(1, TimeUnit.SECONDS); // value has been written to db, read it out and verify. @@ -325,18 +330,22 @@ public void TestUpdateAction() throws Exception { } @Test + @SuppressWarnings("unchecked") public void TestDeleteAction() throws Exception { AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); Foo deleteObj = new Foo(); deleteObj.setField3(5); byte[] deleteBytes = schema.encode(deleteObj); - Message deleteMessage = mock(MessageImpl.class); + Message deleteMessage = mock(MessageImpl.class); CompletableFuture future = new CompletableFuture<>(); - Record deleteRecord = PulsarRecord.builder() + Record deleteRecord = PulsarRecord.builder() .message(deleteMessage) + .schema(autoConsumeSchema) .topicName("fake_topic_name") .ackFunction(() -> future.complete(null)) .build(); @@ -352,7 +361,7 @@ public void TestDeleteAction() throws Exception { deleteMessage.getValue().toString(), deleteRecord.getValue().toString()); - jdbcSink.write(deleteRecord); + jdbcSink.write((Record) deleteRecord); future.get(1, TimeUnit.SECONDS); // value has been written to db, read it out and verify. @@ -848,17 +857,21 @@ public void testNullValueAction(NullValueActionTestConfig config) throws Excepti } } + @SuppressWarnings("unchecked") private Record createMockFooRecord(Foo record, Map actionProperties, CompletableFuture future) { - Message insertMessage = mock(MessageImpl.class); + Message insertMessage = mock(MessageImpl.class); GenericSchema genericAvroSchema; AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).withAlwaysAllowNull(true).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); byte[] insertBytes = schema.encode(record); - Record insertRecord = PulsarRecord.builder() + Record insertRecord = PulsarRecord.builder() .message(insertMessage) .topicName("fake_topic_name") + .schema(autoConsumeSchema) .ackFunction(() -> future.complete(true)) .failFunction(() -> future.complete(false)) .build(); @@ -866,7 +879,7 @@ private Record createMockFooRecord(Foo record, Map) insertRecord; } } diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 105320130055c..31a886db90816 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -175,6 +175,13 @@ test-jar + + org.bouncycastle + bc-fips + ${bouncycastle.bc-fips.version} + test + + org.apache.avro avro diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java index 782f9d5d57dbb..142660813b1a7 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java @@ -27,6 +27,7 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; @@ -44,7 +45,6 @@ import org.apache.pulsar.common.schema.KeyValueEncodingType; import org.apache.pulsar.functions.api.KVRecord; import org.apache.pulsar.functions.api.Record; -import org.apache.pulsar.io.core.PushSource; import org.apache.pulsar.io.core.SourceContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,7 +52,7 @@ /** * Simple Kafka Source to transfer messages from a Kafka topic. */ -public abstract class KafkaAbstractSource extends PushSource { +public abstract class KafkaAbstractSource extends KafkaPushSource { public static final String HEADER_KAFKA_TOPIC_KEY = "__kafka_topic"; public static final String HEADER_KAFKA_PTN_KEY = "__kafka_partition"; public static final String HEADER_KAFKA_OFFSET_KEY = "__kafka_offset"; @@ -63,6 +63,7 @@ public abstract class KafkaAbstractSource extends PushSource { private volatile boolean running = false; private KafkaSourceConfig kafkaSourceConfig; private Thread runnerThread; + private long maxPollIntervalMs; @Override public void open(Map config, SourceContext sourceContext) throws Exception { @@ -126,6 +127,13 @@ public void open(Map config, SourceContext sourceContext) throws props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, kafkaSourceConfig.getAutoOffsetReset()); props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getKeyDeserializationClass()); props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getValueDeserializationClass()); + if (props.containsKey(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG)) { + maxPollIntervalMs = Long.parseLong(props.get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG).toString()); + } else { + maxPollIntervalMs = Long.parseLong( + ConsumerConfig.configDef().defaultValues().get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG) + .toString()); + } try { consumer = new KafkaConsumer<>(beforeCreateConsumer(props)); } catch (Exception ex) { @@ -166,7 +174,7 @@ public void start() { CompletableFuture[] futures = new CompletableFuture[consumerRecords.count()]; int index = 0; for (ConsumerRecord consumerRecord : consumerRecords) { - KafkaRecord record = buildRecord(consumerRecord); + KafkaRecord record = buildRecord(consumerRecord); if (LOG.isDebugEnabled()) { LOG.debug("Write record {} {} {}", record.getKey(), record.getValue(), record.getSchema()); } @@ -175,7 +183,9 @@ public void start() { index++; } if (!kafkaSourceConfig.isAutoCommitEnabled()) { - CompletableFuture.allOf(futures).get(); + // Wait about 2/3 of the time of maxPollIntervalMs. + // so as to avoid waiting for the timeout to be kicked out of the consumer group. + CompletableFuture.allOf(futures).get(maxPollIntervalMs * 2 / 3, TimeUnit.MILLISECONDS); consumer.commitSync(); } } catch (Exception e) { @@ -253,6 +263,21 @@ public void ack() { completableFuture.complete(null); } + @Override + public void fail() { + completableFuture.completeExceptionally( + new RuntimeException( + String.format( + "Failed to process record with kafka topic: %s partition: %d offset: %d key: %s", + record.topic(), + record.partition(), + record.offset(), + getKey() + ) + ) + ); + } + @Override public Schema getSchema() { return schema; diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaPushSource.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaPushSource.java new file mode 100644 index 0000000000000..9319518572183 --- /dev/null +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaPushSource.java @@ -0,0 +1,108 @@ +/* + * 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 org.apache.pulsar.io.kafka; + +import java.util.concurrent.LinkedBlockingQueue; +import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.core.Source; + +/** + * Kafka Push Source. + * To maintain compatibility, we can't pick the PIP-281: https://github.com/apache/pulsar/pull/20807 + * cherry-pick to the historical version, so the class is implemented in the kafka connector. + */ +public abstract class KafkaPushSource implements Source { + + private static class NullRecord implements Record { + @Override + public Object getValue() { + return null; + } + } + + private static class ErrorNotifierRecord implements Record { + private Exception e; + public ErrorNotifierRecord(Exception e) { + this.e = e; + } + @Override + public Object getValue() { + return null; + } + + public Exception getException() { + return e; + } + } + + private LinkedBlockingQueue> queue; + private static final int DEFAULT_QUEUE_LENGTH = 1000; + private final NullRecord nullRecord = new NullRecord(); + + public KafkaPushSource() { + this.queue = new LinkedBlockingQueue<>(this.getQueueLength()); + } + + @Override + public Record read() throws Exception { + Record record = queue.take(); + if (record instanceof ErrorNotifierRecord) { + throw ((ErrorNotifierRecord) record).getException(); + } + if (record instanceof NullRecord) { + return null; + } else { + return record; + } + } + + /** + * Send this message to be written to Pulsar. + * Pass null if you you are done with this task + * @param record next message from source which should be sent to a Pulsar topic + */ + public void consume(Record record) { + try { + if (record != null) { + queue.put(record); + } else { + queue.put(nullRecord); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + /** + * Get length of the queue that records are push onto. + * Users can override this method to customize the queue length + * @return queue length + */ + public int getQueueLength() { + return DEFAULT_QUEUE_LENGTH; + } + + /** + * Allows the source to notify errors asynchronously. + * @param ex + */ + public void notifyError(Exception ex) { + consume(new ErrorNotifierRecord(ex)); + } +} diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java index 7675de0636e8a..6b4719709a178 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java @@ -21,12 +21,18 @@ import com.google.common.collect.ImmutableMap; import java.time.Duration; import java.util.Collections; +import java.util.Arrays; import java.lang.reflect.Field; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.kafka.KafkaAbstractSource; import org.apache.pulsar.io.kafka.KafkaSourceConfig; @@ -46,6 +52,7 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import static org.testng.Assert.fail; @@ -218,6 +225,88 @@ public final void throwExceptionByPoll() throws Exception { source.read(); } + @Test + public final void throwExceptionBySendFail() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 300000); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail + Record record = source.read(); + record.fail(); + + // read again will throw RuntimeException. + try { + source.read(); + fail("Should throw exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertTrue(e.getCause().getMessage().contains("Failed to process record with kafka topic")); + } + } + + @Test + public final void throwExceptionBySendTimeOut() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 1); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail, just read do noting. + source.read(); + + // read again will throw TimeOutException. + try { + source.read(); + fail("Should throw exception"); + } catch (Exception e) { + assertTrue(e instanceof TimeoutException); + } + } + private File getFile(String name) { ClassLoader classLoader = getClass().getClassLoader(); return new File(classLoader.getResource(name).getFile()); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java index 94da382b74dcf..6d558e709716d 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java @@ -148,6 +148,11 @@ public interface MetadataCache { */ void invalidate(String path); + /** + * Force the invalidation of all object in the metadata cache. + */ + void invalidateAll(); + /** * Invalidate and reload an object in the metadata cache. * diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java index 306b6398b5c50..be945d988fb88 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationClient.java @@ -51,11 +51,13 @@ import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Notification; +import org.apache.pulsar.metadata.api.extended.SessionEvent; +import org.apache.pulsar.metadata.impl.AbstractMetadataStore; @Slf4j public class PulsarRegistrationClient implements RegistrationClient { - private final MetadataStore store; + private final AbstractMetadataStore store; private final String ledgersRootPath; // registration paths private final String bookieRegistrationPath; @@ -68,10 +70,11 @@ public class PulsarRegistrationClient implements RegistrationClient { private final Map> writableBookieInfo; private final Map> readOnlyBookieInfo; private final FutureUtil.Sequencer sequencer; + private SessionEvent lastMetadataSessionEvent; public PulsarRegistrationClient(MetadataStore store, String ledgersRootPath) { - this.store = store; + this.store = (AbstractMetadataStore) store; this.ledgersRootPath = ledgersRootPath; this.bookieServiceInfoMetadataCache = store.getMetadataCache(BookieServiceInfoSerde.INSTANCE); this.sequencer = Sequencer.create(); @@ -88,6 +91,7 @@ public PulsarRegistrationClient(MetadataStore store, .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-registration-client")); store.registerListener(this::updatedBookies); + this.store.registerSessionListener(this::refreshBookies); } @Override @@ -95,6 +99,21 @@ public void close() { executor.shutdownNow(); } + private void refreshBookies(SessionEvent sessionEvent) { + lastMetadataSessionEvent = sessionEvent; + if (!SessionEvent.Reconnected.equals(sessionEvent) && !SessionEvent.SessionReestablished.equals(sessionEvent)){ + return; + } + // Clean caches. + store.invalidateCaches(bookieRegistrationPath, bookieAllRegistrationPath, bookieReadonlyRegistrationPath); + bookieServiceInfoMetadataCache.invalidateAll(); + // Refresh caches of the listeners. + getReadOnlyBookies().thenAccept(bookies -> + readOnlyBookiesWatchers.forEach(w -> executor.execute(() -> w.onBookiesChanged(bookies)))); + getWritableBookies().thenAccept(bookies -> + writableBookiesWatchers.forEach(w -> executor.execute(() -> w.onBookiesChanged(bookies)))); + } + @Override public CompletableFuture>> getWritableBookies() { return getBookiesThenFreshCache(bookieRegistrationPath); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java index 4cadf2397a7fa..cc148c2a3117a 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java @@ -23,6 +23,7 @@ import com.github.benmanes.caffeine.cache.AsyncCacheLoader; import com.github.benmanes.caffeine.cache.AsyncLoadingCache; import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.LoadingCache; import com.google.common.annotations.VisibleForTesting; import io.netty.util.concurrent.DefaultThreadFactory; import java.time.Instant; @@ -523,6 +524,13 @@ public void invalidateAll() { existsCache.synchronous().invalidateAll(); } + public void invalidateCaches(String...paths) { + LoadingCache> loadingCache = childrenCache.synchronous(); + for (String path : paths) { + loadingCache.invalidate(path); + } + } + /** * Run the task in the executor thread and fail the future if the executor is shutting down. */ diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index c528ceb2cf5b7..caaa99c5d40cc 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -158,6 +158,7 @@ protected HttpClient createHttpClient() throws ServletException { client.start(); // Content must not be decoded, otherwise the client gets confused. + // Allow encoded content, such as "Content-Encoding: gzip", to pass through without decoding it. client.getContentDecoderFactories().clear(); // Pass traffic to the client, only intercept what's necessary. diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java index b62d988904f63..15163dd7f6063 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java @@ -105,7 +105,7 @@ public void handleLookup(CommandLookupTopic lookup) { log.debug("Lookup Request ID {} from {} rejected - {}.", clientRequestId, clientAddress, throttlingErrorMessage); } - writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady, + writeAndFlush(Commands.newLookupErrorResponse(ServerError.TooManyRequests, throttlingErrorMessage, clientRequestId)); } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index 4ec5b3f77a3e1..1ff141ec13a08 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -398,6 +398,12 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private boolean authenticateMetricsEndpoint = true; + @FieldContext( + category = CATEGORY_HTTP, + doc = "Time in milliseconds that metrics endpoint would time out. Default is 30s.\n" + + " Set it to 0 to disable timeout." + ) + private long metricsServletTimeoutMs = 30000; @FieldContext( category = CATEGORY_SASL_AUTH, diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index b360820ad007c..a3917f2bc6708 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -292,7 +292,8 @@ public void start() throws Exception { } private synchronized void createMetricsServlet() { - this.metricsServlet = new PrometheusMetricsServlet(-1L, proxyConfig.getClusterName()); + this.metricsServlet = + new PrometheusMetricsServlet(proxyConfig.getMetricsServletTimeoutMs(), proxyConfig.getClusterName()); if (pendingMetricsProviders != null) { pendingMetricsProviders.forEach(provider -> metricsServlet.addRawMetricsProvider(provider)); this.pendingMetricsProviders = null; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index e623d4b85aa09..aa612f12b1ba9 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -26,6 +26,7 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.google.common.annotations.VisibleForTesting; +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Gauge; import io.prometheus.client.Gauge.Child; @@ -231,21 +232,36 @@ public void start() throws Exception { if (!metricsInitialized) { // Setup metrics DefaultExports.initialize(); + CollectorRegistry registry = CollectorRegistry.defaultRegistry; // Report direct memory from Netty counters - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesUsed = + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }); + try { + registry.register(jvmMemoryDirectBytesUsed); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_used metric: {}", e.getMessage()); + } - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesMax = + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }); + try { + registry.register(jvmMemoryDirectBytesMax); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_max metric: {}", e.getMessage()); + } metricsInitialized = true; } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java index 4861117ef6ff5..1b63aa14dfe42 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java @@ -20,18 +20,26 @@ import static org.mockito.Mockito.doReturn; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import java.util.Optional; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.BinaryProtoLookupService; +import org.apache.pulsar.client.impl.ClientCnx; +import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.mockito.Mockito; import org.testng.Assert; @@ -112,4 +120,32 @@ public void testLookup() throws Exception { Assert.assertEquals(LookupProxyHandler.REJECTED_PARTITIONS_METADATA_REQUESTS.get(), 5.0d); } + + @Test + public void testLookupThrottling() throws Exception { + PulsarClientImpl client = (PulsarClientImpl) PulsarClient.builder() + .serviceUrl(proxyService.getServiceUrl()).build(); + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + LookupService lookupService = client.getLookup(); + assertTrue(lookupService instanceof BinaryProtoLookupService); + ClientCnx lookupConnection = client.getCnxPool().getConnection(lookupService.resolveHost()).join(); + + // Make no permits to lookup. + Semaphore lookupSemaphore = proxyService.getLookupRequestSemaphore(); + int availablePermits = lookupSemaphore.availablePermits(); + lookupSemaphore.acquire(availablePermits); + + // Verify will receive too many request exception, and the socket will not be closed. + try { + lookupService.getBroker(TopicName.get(tpName)).get(); + fail("Expected too many request error."); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("Too many")); + } + assertTrue(lookupConnection.ctx().channel().isActive()); + + // cleanup. + lookupSemaphore.release(availablePermits); + client.close(); + } } diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 079b033c0942b..03187ff390064 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -430,21 +430,21 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar * Apache Bookkeeper - - bookkeeper-common-4.16.4.jar - - bookkeeper-common-allocator-4.16.4.jar - - bookkeeper-proto-4.16.4.jar - - bookkeeper-server-4.16.4.jar - - bookkeeper-stats-api-4.16.4.jar - - bookkeeper-tools-framework-4.16.4.jar - - circe-checksum-4.16.4.jar - - codahale-metrics-provider-4.16.4.jar - - cpu-affinity-4.16.4.jar - - http-server-4.16.4.jar - - prometheus-metrics-provider-4.16.4.jar - - codahale-metrics-provider-4.16.4.jar - - bookkeeper-slogger-api-4.16.4.jar - - bookkeeper-slogger-slf4j-4.16.4.jar - - native-io-4.16.4.jar + - bookkeeper-common-4.16.5.jar + - bookkeeper-common-allocator-4.16.5.jar + - bookkeeper-proto-4.16.5.jar + - bookkeeper-server-4.16.5.jar + - bookkeeper-stats-api-4.16.5.jar + - bookkeeper-tools-framework-4.16.5.jar + - circe-checksum-4.16.5.jar + - codahale-metrics-provider-4.16.5.jar + - cpu-affinity-4.16.5.jar + - http-server-4.16.5.jar + - prometheus-metrics-provider-4.16.5.jar + - codahale-metrics-provider-4.16.5.jar + - bookkeeper-slogger-api-4.16.5.jar + - bookkeeper-slogger-slf4j-4.16.5.jar + - native-io-4.16.5.jar * Apache Commons - commons-cli-1.5.0.jar - commons-codec-1.15.jar @@ -588,7 +588,7 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.75.jar - - bcprov-ext-jdk18on-1.75.jar - - bcprov-jdk18on-1.75.jar - - bcutil-jdk18on-1.75.jar + - bcpkix-jdk18on-1.78.jar + - bcprov-ext-jdk18on-1.78.jar + - bcprov-jdk18on-1.78.jar + - bcutil-jdk18on-1.78.jar diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java index 5ad1283fe84c4..9c8e2363adf5b 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java @@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.LongAdder; import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.HashingScheme; @@ -103,11 +104,18 @@ public ProducerHandler(WebSocketService service, HttpServletRequest request, Ser request.getRemotePort(), topic); } } catch (Exception e) { - log.warn("[{}:{}] Failed in creating producer on topic {}: {}", request.getRemoteAddr(), - request.getRemotePort(), topic, e.getMessage()); + int errorCode = getErrorCode(e); + boolean isKnownError = errorCode != HttpServletResponse.SC_INTERNAL_SERVER_ERROR; + if (isKnownError) { + log.warn("[{}:{}] Failed in creating producer on topic {}: {}", request.getRemoteAddr(), + request.getRemotePort(), topic, e.getMessage()); + } else { + log.error("[{}:{}] Failed in creating producer on topic {}: {}", request.getRemoteAddr(), + request.getRemotePort(), topic, e.getMessage(), e); + } try { - response.sendError(getErrorCode(e), getErrorMessage(e)); + response.sendError(errorCode, getErrorMessage(e)); } catch (IOException e1) { log.warn("[{}:{}] Failed to send error: {}", request.getRemoteAddr(), request.getRemotePort(), e1.getMessage(), e1); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java index 8d5a896ba4aa9..1ef7b0993de1e 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java @@ -74,9 +74,7 @@ public static void main(String[] args) throws Exception { try { // load config file and start proxy service String configFile = args[0]; - log.info("Loading configuration from {}", configFile); - WebSocketProxyConfiguration config = PulsarConfigurationLoader.create(configFile, - WebSocketProxyConfiguration.class); + WebSocketProxyConfiguration config = loadConfig(configFile); ProxyServer proxyServer = new ProxyServer(config); WebSocketService service = new WebSocketService(config); start(proxyServer, service); @@ -106,6 +104,14 @@ public static void start(ProxyServer proxyServer, WebSocketService service) thro service.start(); } + private static WebSocketProxyConfiguration loadConfig(String configFile) throws Exception { + log.info("Loading configuration from {}", configFile); + WebSocketProxyConfiguration config = PulsarConfigurationLoader.create(configFile, + WebSocketProxyConfiguration.class); + PulsarConfigurationLoader.isComplete(config); + return config; + } + private static final Logger log = LoggerFactory.getLogger(WebSocketServiceStarter.class); } diff --git a/src/rename-netty-native-libs.cmd b/src/rename-netty-native-libs.cmd index 9003f6d0ef499..bfaa16de0812c 100644 --- a/src/rename-netty-native-libs.cmd +++ b/src/rename-netty-native-libs.cmd @@ -42,11 +42,27 @@ call %UNZIP_CMD% cd /d %TMP_DIR%/%FILE_PREFIX% :: Loop through the number of groups -SET Obj_Length=2 +SET Obj_Length=10 SET Obj[0].FROM=libnetty_transport_native_epoll_x86_64.so SET Obj[0].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so -SET Obj[1].FROM=libnetty_tcnative_linux_x86_64.so -SET Obj[1].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so +SET Obj[1].FROM=libnetty_transport_native_epoll_aarch_64.so +SET Obj[1].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so +SET Obj[2].FROM=libnetty_tcnative_linux_x86_64.so +SET Obj[2].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so +SET Obj[3].FROM=libnetty_tcnative_linux_aarch_64.so +SET Obj[3].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so +SET Obj[4].FROM=libnetty_tcnative_osx_x86_64.jnilib +SET Obj[4].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib +SET Obj[5].FROM=libnetty_tcnative_osx_aarch_64.jnilib +SET Obj[5].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib +SET Obj[6].FROM=libnetty_transport_native_io_uring_x86_64.so +SET Obj[6].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so +SET Obj[7].FROM=libnetty_transport_native_io_uring_aarch_64.so +SET Obj[7].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so +SET Obj[8].FROM=libnetty_resolver_dns_native_macos_aarch_64.jnilib +SET Obj[8].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib +SET Obj[9].FROM=libnetty_resolver_dns_native_macos_x86_64.jnilib +SET Obj[9].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib SET Obj_Index=0 :LoopStart diff --git a/src/rename-netty-native-libs.sh b/src/rename-netty-native-libs.sh index 44b971a02c912..ea2a4c0e2421e 100755 --- a/src/rename-netty-native-libs.sh +++ b/src/rename-netty-native-libs.sh @@ -27,7 +27,13 @@ FILE_PREFIX='META-INF/native' FILES_TO_RENAME=( 'libnetty_transport_native_epoll_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so' + 'libnetty_transport_native_epoll_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so' 'libnetty_tcnative_linux_x86_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so' + 'libnetty_tcnative_linux_aarch_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so' + 'libnetty_tcnative_osx_x86_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib' + 'libnetty_tcnative_osx_aarch_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib' + 'libnetty_transport_native_io_uring_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so' + 'libnetty_transport_native_io_uring_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so' 'libnetty_resolver_dns_native_macos_aarch_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib' 'libnetty_resolver_dns_native_macos_x86_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib' ) diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index 349ee615f10c0..ffe79480bcaa6 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -19,7 +19,7 @@ # build go lang examples first in a separate layer -FROM golang:1.15.8 as pulsar-function-go +FROM golang:1.21 as pulsar-function-go COPY target/pulsar-function-go/ /go/src/github.com/apache/pulsar/pulsar-function-go RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go && go install ./...